diff --git a/flink-catalog-aws/flink-catalog-aws-glue/pom.xml b/flink-catalog-aws/flink-catalog-aws-glue/pom.xml
new file mode 100644
index 00000000..db6abb45
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/pom.xml
@@ -0,0 +1,96 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.flink</groupId>
+        <artifactId>flink-catalog-aws-parent</artifactId>
+        <version>4.4-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>flink-catalog-aws-glue</artifactId>
+    <name>Flink : Catalog : AWS : Glue</name>
+
+
+    <packaging>jar</packaging>
+
+    <dependencies>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-api-java</artifactId>
+            <version>${flink.version}</version>
+            <scope>provided</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-aws-base</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>glue</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>apache-client</artifactId>
+        </dependency>
+
+        <!-- ArchUit test dependencies -->
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-architecture-tests-test</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-common</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-api-java</artifactId>
+            <version>${flink.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.projectlombok</groupId>
+            <artifactId>lombok</artifactId>
+            <version>1.18.22</version>
+            <scope>test</scope>
+        </dependency>
+
+    </dependencies>
+
+</project>
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java
new file mode 100644
index 00000000..629dfdab
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalog.java
@@ -0,0 +1,1128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.aws.util.AWSClientUtil;
+import org.apache.flink.connector.aws.util.AWSGeneralUtil;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionImpl;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogView;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedCatalogView;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException;
+import org.apache.flink.table.catalog.exceptions.PartitionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+import org.apache.flink.table.catalog.exceptions.TablePartitionedException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.operator.GlueDatabaseOperator;
+import org.apache.flink.table.catalog.glue.operator.GlueFunctionOperator;
+import org.apache.flink.table.catalog.glue.operator.GluePartitionOperator;
+import org.apache.flink.table.catalog.glue.operator.GlueTableOperator;
+import org.apache.flink.table.catalog.glue.util.GlueUtils;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.functions.FunctionIdentifier;
+import org.apache.flink.util.StringUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.http.apache.ApacheHttpClient;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.GetTablesRequest;
+import software.amazon.awssdk.services.glue.model.GetTablesResponse;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.Partition;
+import software.amazon.awssdk.services.glue.model.Table;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/** Glue catalog implementation that uses AWS Glue Data Catalog as persistence at backend. */
+@PublicEvolving
+public class GlueCatalog extends AbstractCatalog {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueCatalog.class);
+
+    /** instance of GlueOperator to facilitate glue related actions. */
+    public GlueDatabaseOperator glueDatabaseOperator;
+
+    public GlueTableOperator glueTableOperator;
+    public GluePartitionOperator gluePartitionOperator;
+    public GlueFunctionOperator glueFunctionOperator;
+
+    public GlueClient glueClient;
+
+    /** Default database name if not passed as part of catalog. */
+    public static final String DEFAULT_DB = "default";
+
+    public GlueCatalog(
+            String catalogName,
+            String databaseName,
+            ReadableConfig catalogConfig,
+            Properties glueClientProperties) {
+        super(catalogName, databaseName);
+        checkNotNull(catalogConfig, "Catalog config cannot be null.");
+        String glueCatalogId =
+                String.valueOf(
+                        catalogConfig.getOptional(GlueCatalogOptions.GLUE_CATALOG_ID).orElse(null));
+        glueClient = createClient(glueClientProperties);
+        this.glueDatabaseOperator = new GlueDatabaseOperator(getName(), glueClient, glueCatalogId);
+        this.glueTableOperator = new GlueTableOperator(getName(), glueClient, glueCatalogId);
+        this.gluePartitionOperator =
+                new GluePartitionOperator(getName(), glueClient, glueCatalogId);
+        this.glueFunctionOperator = new GlueFunctionOperator(getName(), glueClient, glueCatalogId);
+    }
+
+    private static GlueClient createClient(Properties glueClientProperties) {
+        return AWSClientUtil.createAwsSyncClient(
+                glueClientProperties,
+                AWSGeneralUtil.createSyncHttpClient(
+                        glueClientProperties, ApacheHttpClient.builder()),
+                GlueClient.builder(),
+                GlueCatalogConstants.BASE_GLUE_USER_AGENT_PREFIX_FORMAT,
+                GlueCatalogConstants.GLUE_CLIENT_USER_AGENT_PREFIX);
+    }
+
+    @VisibleForTesting
+    public GlueCatalog(
+            String catalogName,
+            String databaseName,
+            GlueClient glueClient,
+            GlueDatabaseOperator glueDatabaseOperator,
+            GlueTableOperator glueTableOperator,
+            GluePartitionOperator gluePartitionOperator,
+            GlueFunctionOperator glueFunctionOperator) {
+        super(catalogName, databaseName);
+        this.glueClient = glueClient;
+        this.glueDatabaseOperator = glueDatabaseOperator;
+        this.glueTableOperator = glueTableOperator;
+        this.gluePartitionOperator = gluePartitionOperator;
+        this.glueFunctionOperator = glueFunctionOperator;
+    }
+
+    /**
+     * Open the catalog. Used for any required preparation in initialization phase.
+     *
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void open() throws CatalogException {}
+
+    /**
+     * Close the catalog when it is no longer needed and release any resource that it might be
+     * holding.
+     *
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void close() throws CatalogException {
+        try {
+            glueClient.close();
+        } catch (Exception e) {
+            throw new CatalogException("Glue Client is not closed properly!", e);
+        }
+    }
+
+    // ------ databases ------
+
+    /**
+     * Create a database.
+     *
+     * @param databaseName Name of the database to be created
+     * @param database The database definition
+     * @param ignoreIfExists Flag to specify behavior when a database with the given name already
+     *     exists: if set to false, throw a DatabaseAlreadyExistException, if set to true, do
+     *     nothing.
+     * @throws DatabaseAlreadyExistException if the given database already exists and ignoreIfExists
+     *     is false
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void createDatabase(
+            String databaseName, CatalogDatabase database, boolean ignoreIfExists)
+            throws DatabaseAlreadyExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        checkNotNull(database, "Database cannot be null.");
+        databaseName = GlueUtils.getGlueConventionalName(databaseName);
+        if (databaseExists(databaseName) && !ignoreIfExists) {
+            throw new DatabaseAlreadyExistException(getName(), databaseName);
+        } else {
+            glueDatabaseOperator.createGlueDatabase(databaseName, database);
+            LOG.info("Created Database {}.", databaseName);
+        }
+    }
+
+    /**
+     * Drop a database.
+     *
+     * @param databaseName Name of the database to be dropped.
+     * @param ignoreIfNotExists Flag to specify behavior when the database does not exist: if set to
+     *     false, throw an exception, if set to true, do nothing.
+     * @param cascade Flag to specify behavior when the database contains table or function: if set
+     *     to true, delete all tables and functions in the database and then delete the database, if
+     *     set to false, throw an exception.
+     * @throws DatabaseNotExistException if the given database does not exist
+     * @throws DatabaseNotEmptyException if the given database is not empty and isRestrict is true
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void dropDatabase(String databaseName, boolean ignoreIfNotExists, boolean cascade)
+            throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        databaseName = GlueUtils.getGlueConventionalName(databaseName);
+        if (databaseExists(databaseName)) {
+            if (cascade) {
+                List<String> tables = listTables(databaseName);
+                if (!tables.isEmpty()) {
+                    glueDatabaseOperator.deleteTablesFromDatabase(databaseName, tables);
+                    LOG.info("{} Tables deleted from Database {}.", tables.size(), databaseName);
+                }
+                List<String> functions = listFunctions(databaseName);
+                if (!functions.isEmpty()) {
+                    glueDatabaseOperator.deleteFunctionsFromDatabase(databaseName, functions);
+                    LOG.info(
+                            "{} Functions deleted from Database {}.",
+                            functions.size(),
+                            databaseName);
+                }
+            }
+            if (!isDatabaseEmpty(databaseName)) {
+                throw new DatabaseNotEmptyException(getName(), databaseName);
+            }
+            glueDatabaseOperator.dropGlueDatabase(databaseName);
+            LOG.info("Dropped Database: {}.", databaseName);
+        } else if (!ignoreIfNotExists) {
+            throw new DatabaseNotExistException(getName(), databaseName);
+        }
+    }
+
+    /**
+     * Modify existing database.
+     *
+     * @param name Name of the database to be modified
+     * @param newDatabase The new database definition
+     * @param ignoreIfNotExists Flag to specify behavior when the given database does not exist: if
+     *     set to false, throw an exception, if set to true, do nothing.
+     * @throws DatabaseNotExistException if the given database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists)
+            throws DatabaseNotExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(name),
+                "Database name cannot be null or empty.");
+        checkNotNull(newDatabase, "Database cannot be null.");
+        name = GlueUtils.getGlueConventionalName(name);
+        try {
+            CatalogDatabase existingDatabase = glueDatabaseOperator.getDatabase(name);
+            if (existingDatabase != null) {
+                if (existingDatabase.getClass() != newDatabase.getClass()) {
+                    throw new CatalogException(
+                            String.format(
+                                    "Database types don't match. Existing database is '%s' and new database is '%s'.",
+                                    existingDatabase.getClass().getName(),
+                                    newDatabase.getClass().getName()));
+                }
+                glueDatabaseOperator.updateGlueDatabase(name, newDatabase);
+            }
+        } catch (DatabaseNotExistException e) {
+            if (!ignoreIfNotExists) {
+                throw new DatabaseNotExistException(getName(), name);
+            }
+        }
+    }
+
+    /**
+     * Get list of databases in catalog.
+     *
+     * @return a list of the names of all databases
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<String> listDatabases() throws CatalogException {
+        return glueDatabaseOperator.listGlueDatabases();
+    }
+
+    /**
+     * Get a database from this catalog.
+     *
+     * @param databaseName Name of the database
+     * @return The requested database
+     * @throws DatabaseNotExistException if the database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        databaseName = GlueUtils.getGlueConventionalName(databaseName);
+        return glueDatabaseOperator.getDatabase(databaseName);
+    }
+
+    /**
+     * Check if a database exists in this catalog.
+     *
+     * @param databaseName Name of the database
+     * @return true if the given database exists in the catalog false otherwise
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public boolean databaseExists(String databaseName) throws CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        try {
+            return getDatabase(databaseName) != null;
+        } catch (DatabaseNotExistException e) {
+            return false;
+        }
+    }
+
+    /**
+     * Check if database is empty. i.e. it should not contain 1. table 2. functions
+     *
+     * @param databaseName name of database.
+     * @return boolean True/False based on the content of database.
+     * @throws CatalogException Any Exception thrown due to glue error
+     */
+    public boolean isDatabaseEmpty(String databaseName) throws CatalogException {
+        checkArgument(
+                !isNullOrWhitespaceOnly(databaseName), "Database name cannot be null or empty.");
+        GlueUtils.validate(databaseName);
+        GetTablesRequest tablesRequest =
+                GetTablesRequest.builder()
+                        .catalogId(glueTableOperator.getGlueCatalogId())
+                        .databaseName(databaseName)
+                        .maxResults(1)
+                        .build();
+        GetUserDefinedFunctionsRequest functionsRequest =
+                GetUserDefinedFunctionsRequest.builder()
+                        .databaseName(databaseName)
+                        .catalogId(glueFunctionOperator.getGlueCatalogId())
+                        .maxResults(1)
+                        .build();
+        try {
+            GetTablesResponse tableResponse = glueClient.getTables(tablesRequest);
+            GetUserDefinedFunctionsResponse functionResponse =
+                    glueClient.getUserDefinedFunctions(functionsRequest);
+            if (tableResponse.sdkHttpResponse().isSuccessful()
+                    && functionResponse.sdkHttpResponse().isSuccessful()) {
+                return tableResponse.tableList().isEmpty()
+                        && functionResponse.userDefinedFunctions().isEmpty();
+            } else {
+                String errorMessage =
+                        String.format(
+                                "Error checking if database '%s' is empty. Glue API requests failed with the following IDs:\n"
+                                        + "1. GetTables: %s\n"
+                                        + "2. GetUserDefinedFunctions: %s\n"
+                                        + "Please check the Glue service logs for more details.",
+                                databaseName,
+                                tableResponse.responseMetadata().requestId(),
+                                functionResponse.responseMetadata().requestId());
+                throw new CatalogException(errorMessage);
+            }
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    // ------ tables ------
+
+    /**
+     * Creates a new table or view.
+     *
+     * <p>The framework will make sure to call this method with fully validated {@link
+     * ResolvedCatalogTable} or {@link ResolvedCatalogView}. Those instances are easy to serialize
+     * for a durable catalog implementation.
+     *
+     * @param tablePath path of the table or view to be created
+     * @param table the table definition
+     * @param ignoreIfExists flag to specify behavior when a table or view already exists at the
+     *     given path: if set to false, it throws a TableAlreadyExistException, if set to true, do
+     *     nothing.
+     * @throws TableAlreadyExistException if table already exists and ignoreIfExists is false
+     * @throws DatabaseNotExistException if the database in tablePath doesn't exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists)
+            throws TableAlreadyExistException, DatabaseNotExistException, CatalogException {
+        checkNotNull(tablePath, "tablePath cannot be null");
+        checkNotNull(table, "table cannot be null");
+        checkArgument(table instanceof ResolvedCatalogBaseTable, "table should be resolved");
+        if (!databaseExists(tablePath.getDatabaseName())) {
+            throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName());
+        }
+        if (tableExists(tablePath)) {
+            if (!ignoreIfExists) {
+                throw new TableAlreadyExistException(getName(), tablePath);
+            }
+        } else {
+            glueTableOperator.createGlueTable(tablePath, table);
+        }
+    }
+
+    /**
+     * Modifies an existing table or view. Note that the new and old {@link CatalogBaseTable} must
+     * be of the same kind. For example, this doesn't allow altering a regular table to partitioned
+     * table, or altering a view to a table, and vice versa.
+     *
+     * <p>The framework will make sure to call this method with fully validated {@link
+     * ResolvedCatalogTable} or {@link ResolvedCatalogView}. Those instances are easy to serialize
+     * for a durable catalog implementation.
+     *
+     * @param tablePath path of the table or view to be modified
+     * @param newTable the new table definition
+     * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if
+     *     set to false, throw an exception, if set to true, do nothing.
+     * @throws TableNotExistException if the table does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterTable(
+            ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        checkNotNull(newTable, "Table cannot be null.");
+        CatalogBaseTable existingTable = getTable(tablePath);
+        if (existingTable != null) {
+            if (existingTable.getTableKind() != newTable.getTableKind()) {
+                throw new CatalogException(
+                        String.format(
+                                "Table types don't match. Existing table is '%s' and new table is '%s'.",
+                                existingTable.getTableKind(), newTable.getTableKind()));
+            }
+            glueTableOperator.alterGlueTable(tablePath, newTable);
+        } else if (!ignoreIfNotExists) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+    }
+
+    // ------ tables and views ------
+
+    /**
+     * Drop a table or view.
+     *
+     * @param tablePath Path of the table or view to be dropped
+     * @param ignoreIfNotExists Flag to specify behavior when the table or view does not exist: if
+     *     set to false, throw an exception, if set to true, do nothing.
+     * @throws TableNotExistException if the table or view does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        if (tableExists(tablePath)) {
+            glueTableOperator.dropGlueTable(tablePath);
+        } else if (!ignoreIfNotExists) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+    }
+
+    /**
+     * Rename an existing table or view.
+     *
+     * @param tablePath Path of the table or view to be renamed
+     * @param newTableName the new name of the table or view
+     * @param ignoreIfNotExists Flag to specify behavior when the table or view does not exist: if
+     *     set to false, throw an exception, if set to true, do nothing.
+     * @throws TableNotExistException if the table does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists)
+            throws TableNotExistException, TableAlreadyExistException, CatalogException {
+
+        checkNotNull(tablePath, "TablePath cannot be null");
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(newTableName),
+                "Table name cannot be null or empty.");
+
+        if (tableExists(tablePath)) {
+            ObjectPath newTablePath = new ObjectPath(tablePath.getDatabaseName(), newTableName);
+            if (tableExists(newTablePath)) {
+                throw new TableAlreadyExistException(getName(), newTablePath);
+            }
+            glueTableOperator.renameGlueTable(tablePath, newTablePath);
+        } else if (!ignoreIfNotExists) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+    }
+
+    /**
+     * Get names of all tables and views under this database. An empty list is returned if none
+     * exists.
+     *
+     * @param databaseName fully qualified database name.
+     * @return a list of the names of all tables and views in this database
+     * @throws DatabaseNotExistException if the database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<String> listTables(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        if (!databaseExists(databaseName)) {
+            throw new DatabaseNotExistException(getName(), databaseName);
+        }
+        List<String> tableAndViewList =
+                glueTableOperator.getGlueTableList(
+                        databaseName, CatalogBaseTable.TableKind.TABLE.name());
+        tableAndViewList.addAll(listViews(databaseName));
+        return tableAndViewList;
+    }
+
+    /**
+     * Get names of all views under this database. An empty list is returned if none exists.
+     *
+     * @param databaseName the name of the given database
+     * @return a list of the names of all views in the given database
+     * @throws DatabaseNotExistException if the database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<String> listViews(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty");
+
+        if (!databaseExists(databaseName)) {
+            throw new DatabaseNotExistException(getName(), databaseName);
+        }
+        return glueTableOperator.getGlueTableList(
+                databaseName, CatalogBaseTable.TableKind.VIEW.name());
+    }
+
+    /**
+     * Returns a {@link CatalogTable} or {@link CatalogView} identified by the given {@link
+     * ObjectPath}. The framework will resolve the metadata objects when necessary.
+     *
+     * @param tablePath Path of the table or view
+     * @return The requested table or view
+     * @throws TableNotExistException if the target does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogBaseTable getTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        if (!tableExists(tablePath)) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+        return glueTableOperator.getCatalogBaseTableFromGlueTable(
+                glueTableOperator.getGlueTable(tablePath));
+    }
+
+    /**
+     * Check if a table or view exists in this catalog.
+     *
+     * @param tablePath Path of the table or view
+     * @return true if the given table exists in the catalog false otherwise
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public boolean tableExists(ObjectPath tablePath) throws CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        return databaseExists(tablePath.getDatabaseName())
+                && glueTableOperator.glueTableExists(tablePath);
+    }
+
+    // ------ functions ------
+    /**
+     * Create a function. Function name should be handled in a case-insensitive way.
+     *
+     * @param path path of the function
+     * @param function the function to be created
+     * @param ignoreIfExists flag to specify behavior if a function with the given name already
+     *     exists: if set to false, it throws a FunctionAlreadyExistException, if set to true,
+     *     nothing happens.
+     * @throws FunctionAlreadyExistException if the function already exist
+     * @throws DatabaseNotExistException if the given database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void createFunction(ObjectPath path, CatalogFunction function, boolean ignoreIfExists)
+            throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException {
+        checkNotNull(path, "Function path cannot be null.");
+        checkNotNull(function, "Catalog Function cannot be null.");
+        ObjectPath functionPath = normalize(path);
+        if (!databaseExists(functionPath.getDatabaseName())) {
+            throw new DatabaseNotExistException(getName(), functionPath.getDatabaseName());
+        }
+        if (!functionExists(functionPath)) {
+            glueFunctionOperator.createGlueFunction(functionPath, function);
+        } else {
+            if (!ignoreIfExists) {
+                throw new FunctionAlreadyExistException(getName(), functionPath);
+            }
+        }
+    }
+
+    public ObjectPath normalize(ObjectPath path) {
+        return new ObjectPath(
+                path.getDatabaseName(), FunctionIdentifier.normalizeName(path.getObjectName()));
+    }
+
+    /**
+     * Modify an existing function. Function name should be handled in a case-insensitive way.
+     *
+     * @param path path of the function
+     * @param newFunction the function to be modified
+     * @param ignoreIfNotExists flag to specify behavior if the function does not exist: if set to
+     *     false, throw an exception if set to true, nothing happens
+     * @throws FunctionNotExistException if the function does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterFunction(
+            ObjectPath path, CatalogFunction newFunction, boolean ignoreIfNotExists)
+            throws FunctionNotExistException, CatalogException {
+        checkNotNull(path, "Function path cannot be null.");
+        checkNotNull(newFunction, "Catalog Function cannot be null.");
+        ObjectPath functionPath = normalize(path);
+        CatalogFunction existingFunction = getFunction(functionPath);
+        if (existingFunction != null) {
+            if (existingFunction.getClass() != newFunction.getClass()) {
+                throw new CatalogException(
+                        String.format(
+                                "Function types don't match. Existing function is '%s' and new function is '%s'.",
+                                existingFunction.getClass().getName(),
+                                newFunction.getClass().getName()));
+            }
+            glueFunctionOperator.alterGlueFunction(functionPath, newFunction);
+        } else if (!ignoreIfNotExists) {
+            throw new FunctionNotExistException(getName(), functionPath);
+        }
+    }
+
+    /**
+     * Drop a function. Function name should be handled in a case-insensitive way.
+     *
+     * @param path path of the function to be dropped
+     * @param ignoreIfNotExists flag to specify behavior if the function does not exist: if set to
+     *     false, throw an exception if set to true, nothing happens
+     * @throws FunctionNotExistException if the function does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void dropFunction(ObjectPath path, boolean ignoreIfNotExists)
+            throws FunctionNotExistException, CatalogException {
+        checkNotNull(path, "Function path cannot be null.");
+        ObjectPath functionPath = normalize(path);
+        if (functionExists(functionPath)) {
+            glueFunctionOperator.dropGlueFunction(functionPath);
+        } else if (!ignoreIfNotExists) {
+            throw new FunctionNotExistException(getName(), functionPath);
+        }
+    }
+
+    /**
+     * List the names of all functions in the given database. An empty list is returned if none is
+     * registered.
+     *
+     * @param databaseName name of the database.
+     * @return a list of the names of the functions in this database
+     * @throws DatabaseNotExistException if the database does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<String> listFunctions(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(databaseName),
+                "Database name cannot be null or empty.");
+        databaseName = GlueUtils.getGlueConventionalName(databaseName);
+        if (!databaseExists(databaseName)) {
+            throw new DatabaseNotExistException(getName(), databaseName);
+        }
+        return glueFunctionOperator.listGlueFunctions(databaseName);
+    }
+
+    /**
+     * Get the function. Function name should be handled in a case-insensitive way.
+     *
+     * @param path path of the function
+     * @return the requested function
+     * @throws FunctionNotExistException if the function does not exist in the catalog
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogFunction getFunction(ObjectPath path)
+            throws FunctionNotExistException, CatalogException {
+        checkNotNull(path, "Function path cannot be null.");
+        ObjectPath functionPath = normalize(path);
+        if (!functionExists(functionPath)) {
+            throw new FunctionNotExistException(getName(), functionPath);
+        } else {
+            return glueFunctionOperator.getGlueFunction(functionPath);
+        }
+    }
+
+    /**
+     * Check whether a function exists or not. Function name should be handled in a case-insensitive
+     * way.
+     *
+     * @param path path of the function
+     * @return true if the function exists in the catalog false otherwise
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public boolean functionExists(ObjectPath path) throws CatalogException {
+        checkNotNull(path, "Function path cannot be null.");
+        ObjectPath functionPath = normalize(path);
+        return databaseExists(functionPath.getDatabaseName())
+                && glueFunctionOperator.glueFunctionExists(functionPath);
+    }
+
+    // ------ partitions ------
+    /**
+     * Create a partition.
+     *
+     * @param tablePath path of the table.
+     * @param partitionSpec partition spec of the partition
+     * @param partition the partition to add.
+     * @param ignoreIfExists flag to specify behavior if a table with the given name already exists:
+     *     if set to false, it throws a TableAlreadyExistException, if set to true, nothing happens.
+     * @throws TableNotExistException thrown if the target table does not exist
+     * @throws TableNotPartitionedException thrown if the target table is not partitioned
+     * @throws PartitionSpecInvalidException thrown if the given partition spec is invalid
+     * @throws PartitionAlreadyExistsException thrown if the target partition already exists
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void createPartition(
+            ObjectPath tablePath,
+            CatalogPartitionSpec partitionSpec,
+            CatalogPartition partition,
+            boolean ignoreIfExists)
+            throws TableNotExistException, TableNotPartitionedException,
+                    PartitionSpecInvalidException, PartitionAlreadyExistsException,
+                    CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        checkNotNull(partitionSpec, "PartitionSpec cannot be null.");
+        checkNotNull(partition, "Partition cannot be null.");
+        Table glueTable = glueTableOperator.getGlueTable(tablePath);
+        gluePartitionOperator.ensurePartitionedTable(tablePath, glueTable);
+        if (!partitionExists(tablePath, partitionSpec)) {
+            gluePartitionOperator.createGluePartition(glueTable, partitionSpec, partition);
+        } else {
+            if (!ignoreIfExists) {
+                throw new PartitionAlreadyExistsException(getName(), tablePath, partitionSpec);
+            }
+        }
+    }
+
+    /**
+     * Get CatalogPartitionSpec of all partitions of the table.
+     *
+     * @param tablePath path of the table
+     * @return a list of CatalogPartitionSpec of the table
+     * @throws TableNotExistException thrown if the table does not exist in the catalog
+     * @throws TableNotPartitionedException thrown if the table is not partitioned
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath)
+            throws TableNotExistException, TableNotPartitionedException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        if (!tableExists(tablePath)) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+        if (isPartitionedTable(tablePath)) {
+            return gluePartitionOperator.listPartitions(tablePath);
+        }
+        throw new TableNotPartitionedException(getName(), tablePath);
+    }
+
+    public boolean isPartitionedTable(ObjectPath tablePath) {
+        CatalogBaseTable table;
+        try {
+            table = getTable(tablePath);
+            if (table instanceof CatalogTable) {
+                CatalogTable catalogTable = (CatalogTable) table;
+                return catalogTable.isPartitioned();
+            }
+            return false;
+        } catch (TableNotExistException e) {
+            throw new CatalogException(tablePath.getFullName() + " Table doesn't Exists.", e);
+        }
+    }
+
+    /**
+     * Get CatalogPartitionSpec of all partitions that is under the given CatalogPartitionSpec in
+     * the table.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec the partition spec to list
+     * @return a list of CatalogPartitionSpec that is under the given CatalogPartitionSpec in the
+     *     table
+     * @throws TableNotExistException thrown if the table does not exist in the catalog
+     * @throws TableNotPartitionedException thrown if the table is not partitioned
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<CatalogPartitionSpec> listPartitions(
+            ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+            throws TableNotExistException, TableNotPartitionedException,
+                    PartitionSpecInvalidException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        checkNotNull(partitionSpec, "Partition spec cannot be null.");
+        if (!tableExists(tablePath)) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+        if (!isPartitionedTable(tablePath)) {
+            throw new TableNotPartitionedException(getName(), tablePath);
+        }
+        return gluePartitionOperator.listPartitions(tablePath, partitionSpec);
+    }
+
+    /**
+     * Get CatalogPartitionSpec of partitions by expression filters in the table.
+     *
+     * <p>NOTE: For FieldReferenceExpression, the field index is based on schema of this table
+     * instead of partition columns only.
+     *
+     * <p>The passed in predicates have been translated in conjunctive form.
+     *
+     * <p>If catalog does not support this interface at present, throw an {@link
+     * UnsupportedOperationException} directly. If the catalog does not have a valid filter, throw
+     * the {@link UnsupportedOperationException} directly. Planner will fall back to get all
+     * partitions and filter by itself.
+     *
+     * @param tablePath path of the table
+     * @param filters filters to push down filter to catalog
+     * @return a list of CatalogPartitionSpec that is under the given CatalogPartitionSpec in the
+     *     table
+     * @throws TableNotExistException thrown if the table does not exist in the catalog
+     * @throws TableNotPartitionedException thrown if the table is not partitioned
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public List<CatalogPartitionSpec> listPartitionsByFilter(
+            ObjectPath tablePath, List<Expression> filters)
+            throws TableNotExistException, TableNotPartitionedException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        if (!tableExists(tablePath)) {
+            throw new TableNotExistException(getName(), tablePath);
+        }
+        if (!isPartitionedTable(tablePath)) {
+            throw new TableNotPartitionedException(getName(), tablePath);
+        }
+        return gluePartitionOperator.listGluePartitionsByFilter(tablePath, filters);
+    }
+
+    /**
+     * Get a partition of the given table. The given partition spec keys and values need to be
+     * matched exactly for a result.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of partition to get
+     * @return the requested partition
+     * @throws PartitionNotExistException thrown if the partition doesn't exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+            throws PartitionNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        checkNotNull(partitionSpec, "CatalogPartitionSpec cannot be null.");
+        Table glueTable;
+        try {
+            glueTable = glueTableOperator.getGlueTable(tablePath);
+        } catch (TableNotExistException e) {
+            throw new CatalogException("Table doesn't exist in Glue Data Catalog", e);
+        }
+        Partition gluePartition = gluePartitionOperator.getGluePartition(glueTable, partitionSpec);
+        if (gluePartition == null) {
+            throw new PartitionNotExistException(getName(), tablePath, partitionSpec);
+        }
+        Map<String, String> catalogPartitionProperties =
+                new HashMap<>(gluePartition.storageDescriptor().parameters());
+        String comment = catalogPartitionProperties.remove(GlueCatalogConstants.COMMENT);
+        return new CatalogPartitionImpl(catalogPartitionProperties, comment);
+    }
+
+    /**
+     * Check whether a partition exists or not.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition to check
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+            throws CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null");
+        if (!databaseExists(tablePath.getDatabaseName())) {
+            throw new CatalogException(tablePath.getDatabaseName() + " Database doesn't exists.");
+        }
+        try {
+            Table glueTable = glueTableOperator.getGlueTable(tablePath);
+            return gluePartitionOperator.gluePartitionExists(tablePath, glueTable, partitionSpec);
+        } catch (TableNotExistException e) {
+            throw new CatalogException(
+                    tablePath.getFullName() + " Table doesn't Exists in Glue Data Catalog.", e);
+        }
+    }
+
+    /**
+     * Drop a partition.
+     *
+     * @param tablePath path of the table.
+     * @param partitionSpec partition spec of the partition to drop
+     * @param ignoreIfNotExists flag to specify behavior if the database does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws PartitionNotExistException thrown if the target partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void dropPartition(
+            ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists)
+            throws PartitionNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        checkNotNull(partitionSpec, "PartitionSpec cannot be null.");
+        if (partitionExists(tablePath, partitionSpec)) {
+            Table glueTable;
+            try {
+                glueTable = glueTableOperator.getGlueTable(tablePath);
+            } catch (TableNotExistException e) {
+                throw new CatalogException(tablePath.getFullName() + " Table doesn't exists.", e);
+            }
+            gluePartitionOperator.dropGluePartition(tablePath, partitionSpec, glueTable);
+        } else if (!ignoreIfNotExists) {
+            throw new PartitionNotExistException(getName(), tablePath, partitionSpec);
+        }
+    }
+
+    /**
+     * Alter a partition.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition
+     * @param newPartition new partition to replace the old one
+     * @param ignoreIfNotExists flag to specify behavior if the database does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws PartitionNotExistException thrown if the target partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterPartition(
+            ObjectPath tablePath,
+            CatalogPartitionSpec partitionSpec,
+            CatalogPartition newPartition,
+            boolean ignoreIfNotExists)
+            throws PartitionNotExistException, CatalogException {
+        checkNotNull(tablePath, "TablePath cannot be null.");
+        checkNotNull(partitionSpec, "CatalogPartitionSpec cannot be null.");
+        checkNotNull(newPartition, "New partition cannot be null.");
+        CatalogPartition existingPartition = getPartition(tablePath, partitionSpec);
+        if (existingPartition != null) {
+            try {
+                Table glueTable = glueTableOperator.getGlueTable(tablePath);
+                gluePartitionOperator.alterGluePartition(
+                        tablePath, glueTable, partitionSpec, newPartition);
+            } catch (TableNotExistException e) {
+                throw new CatalogException("Table Not Found in Glue data catalog", e);
+            } catch (PartitionSpecInvalidException e) {
+                throw new CatalogException("Invalid Partition Spec", e);
+            }
+        } else if (!ignoreIfNotExists) {
+            throw new PartitionNotExistException(getName(), tablePath, partitionSpec);
+        }
+    }
+
+    /**
+     * Get the statistics of a table.
+     *
+     * @param tablePath path of the table
+     * @return statistics of the given table
+     * @throws TableNotExistException if the table does not exist in the catalog
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogTableStatistics getTableStatistics(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return CatalogTableStatistics.UNKNOWN;
+    }
+
+    /**
+     * Get the column statistics of a table.
+     *
+     * @param tablePath path of the table
+     * @return column statistics of the given table
+     * @throws TableNotExistException if the table does not exist in the catalog
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+        return CatalogColumnStatistics.UNKNOWN;
+    }
+
+    /**
+     * Get the statistics of a partition.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition
+     * @return statistics of the given partition
+     * @throws PartitionNotExistException if the partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogTableStatistics getPartitionStatistics(
+            ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+            throws PartitionNotExistException, CatalogException {
+        return CatalogTableStatistics.UNKNOWN;
+    }
+
+    /**
+     * Get the column statistics of a partition.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition
+     * @return column statistics of the given partition
+     * @throws PartitionNotExistException if the partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public CatalogColumnStatistics getPartitionColumnStatistics(
+            ObjectPath tablePath, CatalogPartitionSpec partitionSpec)
+            throws PartitionNotExistException, CatalogException {
+        return CatalogColumnStatistics.UNKNOWN;
+    }
+
+    /**
+     * Update the statistics of a table.
+     *
+     * @param tablePath path of the table
+     * @param tableStatistics new statistics to update
+     * @param ignoreIfNotExists flag to specify behavior if the table does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws TableNotExistException if the table does not exist in the catalog
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterTableStatistics(
+            ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException {
+        throw new UnsupportedOperationException("Operation with Statistics not supported.");
+    }
+
+    /**
+     * Update the column statistics of a table.
+     *
+     * @param tablePath path of the table
+     * @param columnStatistics new column statistics to update
+     * @param ignoreIfNotExists flag to specify behavior if the table does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws TableNotExistException if the table does not exist in the catalog
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterTableColumnStatistics(
+            ObjectPath tablePath,
+            CatalogColumnStatistics columnStatistics,
+            boolean ignoreIfNotExists)
+            throws TableNotExistException, CatalogException, TablePartitionedException {
+        throw new UnsupportedOperationException("Operation with Statistics not supported.");
+    }
+
+    /**
+     * Update the statistics of a table partition.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition
+     * @param partitionStatistics new statistics to update
+     * @param ignoreIfNotExists flag to specify behavior if the partition does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws PartitionNotExistException if the partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterPartitionStatistics(
+            ObjectPath tablePath,
+            CatalogPartitionSpec partitionSpec,
+            CatalogTableStatistics partitionStatistics,
+            boolean ignoreIfNotExists)
+            throws PartitionNotExistException, CatalogException {
+        throw new UnsupportedOperationException("Operation with Statistics not supported.");
+    }
+
+    /**
+     * Update the column statistics of a table partition.
+     *
+     * @param tablePath path of the table
+     * @param partitionSpec partition spec of the partition @@param columnStatistics new column
+     *     statistics to update
+     * @param columnStatistics column related statistics
+     * @param ignoreIfNotExists flag to specify behavior if the partition does not exist: if set to
+     *     false, throw an exception, if set to true, nothing happens.
+     * @throws PartitionNotExistException if the partition does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    @Override
+    public void alterPartitionColumnStatistics(
+            ObjectPath tablePath,
+            CatalogPartitionSpec partitionSpec,
+            CatalogColumnStatistics columnStatistics,
+            boolean ignoreIfNotExists)
+            throws PartitionNotExistException, CatalogException {
+        throw new UnsupportedOperationException("Operation with Statistics not supported.");
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java
new file mode 100644
index 00000000..254557be
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/GlueCatalogOptions.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import org.apache.flink.table.catalog.CommonCatalogOptions;
+import org.apache.flink.table.catalog.glue.constants.AWSGlueConfigConstants;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+
+import software.amazon.awssdk.regions.Region;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalog.DEFAULT_DB;
+
+/** Collection of {@link ConfigOption} used in GlueCatalog. */
+@Internal
+public class GlueCatalogOptions extends CommonCatalogOptions {
+
+    public static final String IDENTIFIER = "glue";
+    public static final ConfigOption<String> DEFAULT_DATABASE =
+            ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY)
+                    .stringType()
+                    .defaultValue(DEFAULT_DB);
+
+    public static final ConfigOption<String> INPUT_FORMAT =
+            ConfigOptions.key(GlueCatalogConstants.TABLE_INPUT_FORMAT)
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<String> OUTPUT_FORMAT =
+            ConfigOptions.key(GlueCatalogConstants.TABLE_OUTPUT_FORMAT)
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<String> GLUE_CATALOG_ENDPOINT =
+            ConfigOptions.key(AWSGlueConfigConstants.GLUE_CATALOG_ENDPOINT)
+                    .stringType()
+                    .noDefaultValue();
+
+    public static final ConfigOption<String> GLUE_CATALOG_ID =
+            ConfigOptions.key(AWSGlueConfigConstants.GLUE_CATALOG_ID).stringType().noDefaultValue();
+
+    public static final ConfigOption<String> GLUE_ACCOUNT_ID =
+            ConfigOptions.key(AWSGlueConfigConstants.GLUE_ACCOUNT_ID).stringType().noDefaultValue();
+
+    public static final ConfigOption<String> CREDENTIAL_PROVIDER =
+            ConfigOptions.key(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER)
+                    .stringType()
+                    .defaultValue(String.valueOf(AWSConfigConstants.CredentialProvider.AUTO));
+
+    public static final ConfigOption<String> HTTP_CLIENT_TYPE =
+            ConfigOptions.key(AWSConfigConstants.HTTP_CLIENT_TYPE)
+                    .stringType()
+                    .defaultValue(AWSConfigConstants.CLIENT_TYPE_APACHE);
+
+    public static final ConfigOption<String> REGION =
+            ConfigOptions.key(AWSConfigConstants.AWS_REGION)
+                    .stringType()
+                    .defaultValue(Region.US_WEST_1.toString());
+
+    public static Set<ConfigOption<?>> getAllConfigOptions() {
+        Set<ConfigOption<?>> configOptions = new HashSet<>();
+        configOptions.add(INPUT_FORMAT);
+        configOptions.add(OUTPUT_FORMAT);
+        configOptions.add(GLUE_CATALOG_ENDPOINT);
+        configOptions.add(GLUE_ACCOUNT_ID);
+        configOptions.add(GLUE_CATALOG_ID);
+        configOptions.add(DEFAULT_DATABASE);
+        configOptions.add(HTTP_CLIENT_TYPE);
+        configOptions.add(REGION);
+        configOptions.add(CREDENTIAL_PROVIDER);
+        return configOptions;
+    }
+
+    public static Set<ConfigOption<?>> getRequiredConfigOptions() {
+        return new HashSet<>();
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java
new file mode 100644
index 00000000..5eeee652
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/TypeMapper.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.AbstractDataType;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+/**
+ * The {@code TypeMapper} class provides utility methods to map Flink's {@link LogicalType} to AWS.
+ * Glue data types and vice versa.
+ *
+ * <p>This class supports conversion between Flink's logical types and Glue data types, handling
+ * both primitive types and complex types such as arrays, maps, and rows. The mapping allows for
+ * seamless integration between Flink and AWS Glue, enabling Flink to read from and write to Glue
+ * tables with the appropriate data types.
+ *
+ * <p>For complex types like arrays, maps, and rows, the conversion is handled recursively, ensuring
+ * that nested types are also converted accurately.
+ *
+ * <p>This class currently supports the following mappings:
+ *
+ * <ul>
+ *   <li>Flink {@code IntType} -> Glue {@code int}
+ *   <li>Flink {@code BigIntType} -> Glue {@code bigint}
+ *   <li>Flink {@code VarCharType} -> Glue {@code string}
+ *   <li>Flink {@code BooleanType} -> Glue {@code boolean}
+ *   <li>Flink {@code DecimalType} -> Glue {@code decimal}
+ *   <li>Flink {@code FloatType} -> Glue {@code float}
+ *   <li>Flink {@code DoubleType} -> Glue {@code double}
+ *   <li>Flink {@code DateType} -> Glue {@code date}
+ *   <li>Flink {@code TimestampType} -> Glue {@code timestamp}
+ *   <li>Flink {@code ArrayType} -> Glue {@code array<elementType>}
+ *   <li>Flink {@code MapType} -> Glue {@code map<keyType,valueType>}
+ *   <li>Flink {@code RowType} -> Glue {@code struct<fieldName:fieldType, ...>}
+ * </ul>
+ *
+ * <p>Note: Struct type handling in {@code glueTypeToFlinkType} is currently not supported and will
+ * throw an {@link UnsupportedOperationException}.
+ *
+ * @see org.apache.flink.table.types.logical.LogicalType
+ * @see org.apache.flink.table.api.DataTypes
+ * @see org.apache.flink.table.catalog.CatalogTable
+ * @see org.apache.flink.table.catalog.ResolvedCatalogTable
+ */
+public class TypeMapper {
+
+    /**
+     * Maps a given Flink {@link LogicalType} to its corresponding AWS Glue data type as a string.
+     *
+     * @param logicalType the Flink logical type to be mapped
+     * @return the corresponding AWS Glue data type as a string
+     * @throws UnsupportedOperationException if the Flink type is not supported
+     */
+    public static String mapFlinkTypeToGlueType(LogicalType logicalType) {
+        if (logicalType instanceof IntType) {
+            return "int";
+        } else if (logicalType instanceof BigIntType) {
+            return "bigint";
+        } else if (logicalType instanceof VarCharType) {
+            return "string";
+        } else if (logicalType instanceof BooleanType) {
+            return "boolean";
+        } else if (logicalType instanceof DecimalType) {
+            return "decimal";
+        } else if (logicalType instanceof FloatType) {
+            return "float";
+        } else if (logicalType instanceof DoubleType) {
+            return "double";
+        } else if (logicalType instanceof DateType) {
+            return "date";
+        } else if (logicalType instanceof TimestampType) {
+            return "timestamp";
+        } else if (logicalType instanceof ArrayType) {
+            ArrayType arrayType = (ArrayType) logicalType;
+            String elementType = mapFlinkTypeToGlueType(arrayType.getElementType());
+            return "array<" + elementType + ">";
+        } else if (logicalType instanceof MapType) {
+            MapType mapType = (MapType) logicalType;
+            String keyType = mapFlinkTypeToGlueType(mapType.getKeyType());
+            String valueType = mapFlinkTypeToGlueType(mapType.getValueType());
+            return "map<" + keyType + "," + valueType + ">";
+        } else if (logicalType instanceof RowType) {
+            RowType rowType = (RowType) logicalType;
+            StringBuilder structType = new StringBuilder("struct<");
+            for (RowType.RowField field : rowType.getFields()) {
+                structType
+                        .append(field.getName())
+                        .append(":")
+                        .append(mapFlinkTypeToGlueType(field.getType()))
+                        .append(",");
+            }
+            // Remove the trailing comma and close the struct definition
+            structType.setLength(structType.length() - 1);
+            structType.append(">");
+            return structType.toString();
+        } else {
+            throw new UnsupportedOperationException("Unsupported Flink type: " + logicalType);
+        }
+    }
+
+    /**
+     * Maps a given AWS Glue data type as a string to its corresponding Flink {@link
+     * AbstractDataType}.
+     *
+     * @param glueType the AWS Glue data type as a string
+     * @return the corresponding Flink data type
+     * @throws UnsupportedOperationException if the Glue type is not supported
+     */
+    public static AbstractDataType<?> glueTypeToFlinkType(String glueType) {
+        if (glueType.equals("int")) {
+            return DataTypes.INT();
+        } else if (glueType.equals("bigint")) {
+            return DataTypes.BIGINT();
+        } else if (glueType.equals("string")) {
+            return DataTypes.STRING();
+        } else if (glueType.equals("boolean")) {
+            return DataTypes.BOOLEAN();
+        } else if (glueType.equals("decimal")) {
+            return DataTypes.DECIMAL(10, 0);
+        } else if (glueType.equals("float")) {
+            return DataTypes.FLOAT();
+        } else if (glueType.equals("double")) {
+            return DataTypes.DOUBLE();
+        } else if (glueType.equals("date")) {
+            return DataTypes.DATE();
+        } else if (glueType.equals("timestamp")) {
+            return DataTypes.TIMESTAMP(5);
+        } else if (glueType.startsWith("array")) {
+            String elementType = glueType.substring(6, glueType.length() - 1);
+            return DataTypes.ARRAY(glueTypeToFlinkType(elementType));
+        } else if (glueType.startsWith("map")) {
+            // Example: map<string, string> -> DataTypes.MAP(DataTypes.STRING(),
+            // DataTypes.STRING())
+            int commaIndex = glueType.indexOf(",");
+            String keyType = glueType.substring(4, commaIndex);
+            String valueType = glueType.substring(commaIndex + 1, glueType.length() - 1);
+            return DataTypes.MAP(glueTypeToFlinkType(keyType), glueTypeToFlinkType(valueType));
+        } else {
+            // Handle struct type if necessary
+            // For this case, custom parsing might be required based on struct definition
+            throw new UnsupportedOperationException("Struct type not yet supported");
+        }
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java
new file mode 100644
index 00000000..e2c992a5
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstants.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.constants;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+/** Configuration keys for AWS Glue Data Catalog service usage. */
+@PublicEvolving
+public class AWSGlueConfigConstants {
+
+    /**
+     * Configure an alternative endpoint of the Glue service for GlueCatalog to access.
+     *
+     * <p>This could be used to use GlueCatalog with any glue-compatible metastore service that has
+     * a different endpoint
+     */
+    public static final String GLUE_CATALOG_ENDPOINT = "aws.glue.endpoint";
+
+    /**
+     * The ID of the Glue Data Catalog where the tables reside. If none is provided, Glue
+     * automatically uses the caller's AWS account ID by default.
+     *
+     * <p>For more details, see <a
+     * href="https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html">...</a>
+     */
+    public static final String GLUE_CATALOG_ID = "aws.glue.id";
+
+    /**
+     * The account ID used in a Glue resource ARN, e.g.
+     * arn:aws:glue:us-east-1:1000000000000:table/db1/table1
+     */
+    public static final String GLUE_ACCOUNT_ID = "aws.glue.account-id";
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java
new file mode 100644
index 00000000..61ee8370
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/constants/GlueCatalogConstants.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.constants;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.glue.GlueCatalog;
+
+import java.util.regex.Pattern;
+
+/** Constants and Defined Values used for {@link GlueCatalog}. */
+@Internal
+public class GlueCatalogConstants {
+    public static final String COMMENT = "comment";
+    public static final String DEFAULT_SEPARATOR = ":";
+    public static final String LOCATION_SEPARATOR = "/";
+    public static final String LOCATION_URI = "path";
+    public static final String AND = "and";
+    public static final String NEXT_LINE = "\n";
+    public static final String SPACE = " ";
+
+    public static final String TABLE_OWNER = "owner";
+    public static final String TABLE_INPUT_FORMAT = "table.input.format";
+    public static final String TABLE_OUTPUT_FORMAT = "table.output.format";
+
+    public static final String FLINK_SCALA_FUNCTION_PREFIX = "flink:scala:";
+    public static final String FLINK_PYTHON_FUNCTION_PREFIX = "flink:python:";
+    public static final String FLINK_JAVA_FUNCTION_PREFIX = "flink:java:";
+
+    public static final String FLINK_CATALOG = "FLINK_CATALOG";
+
+    public static final Pattern GLUE_DB_PATTERN = Pattern.compile("^[a-z0-9_]{1,255}$");
+    public static final String GLUE_EXCEPTION_MSG_IDENTIFIER = "GLUE EXCEPTION";
+    public static final String TABLE_NOT_EXISTS_IDENTIFIER = "TABLE DOESN'T EXIST";
+    public static final String DEFAULT_PARTITION_NAME = "__GLUE_DEFAULT_PARTITION__";
+
+    public static final int UDF_CLASS_NAME_SIZE = 3;
+
+    public static final String BASE_GLUE_USER_AGENT_PREFIX_FORMAT =
+            "Apache Flink %s (%s) Glue Catalog";
+
+    /** Glue Catalog identifier for user agent prefix. */
+    public static final String GLUE_CLIENT_USER_AGENT_PREFIX = "aws.glue.client.user-agent-prefix";
+
+    public static final String IS_PERSISTED = "isPersisted";
+    public static final String EXPLAIN_EXTRAS = "explainExtras";
+    public static final String IS_PHYSICAL = "isPhysical";
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java
new file mode 100644
index 00000000..c5eb68be
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactory.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.factory;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.glue.GlueCatalog;
+import org.apache.flink.table.catalog.glue.GlueCatalogOptions;
+import org.apache.flink.table.catalog.glue.util.GlueCatalogOptionsUtils;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Catalog factory for {@link GlueCatalog}. */
+@PublicEvolving
+public class GlueCatalogFactory implements CatalogFactory {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueCatalogFactory.class);
+
+    @Override
+    public String factoryIdentifier() {
+        return GlueCatalogOptions.IDENTIFIER;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        Set<ConfigOption<?>> allConfigs = GlueCatalogOptions.getAllConfigOptions();
+        allConfigs.removeAll(GlueCatalogOptions.getRequiredConfigOptions());
+        return allConfigs;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        return GlueCatalogOptions.getRequiredConfigOptions();
+    }
+
+    @Override
+    public Catalog createCatalog(Context context) {
+        final FactoryUtil.CatalogFactoryHelper helper =
+                FactoryUtil.createCatalogFactoryHelper(this, context);
+        GlueCatalogOptionsUtils optionsUtils =
+                new GlueCatalogOptionsUtils(context.getOptions(), context.getConfiguration());
+        helper.validateExcept(optionsUtils.getNonValidatedPrefixes().toArray(new String[0]));
+        if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                    context.getOptions().entrySet().stream()
+                            .map(entry -> entry.getKey() + "-> " + entry.getValue())
+                            .collect(Collectors.joining("\n")));
+        }
+        Properties glueCatalogValidatedProperties = optionsUtils.getValidatedConfigurations();
+        return new GlueCatalog(
+                context.getName(),
+                helper.getOptions().get(GlueCatalogOptions.DEFAULT_DATABASE),
+                context.getConfiguration(),
+                glueCatalogValidatedProperties);
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java
new file mode 100644
index 00000000..710ce3f3
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueDatabaseOperator.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.operator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.util.GlueUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.AlreadyExistsException;
+import software.amazon.awssdk.services.glue.model.BatchDeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.BatchDeleteTableResponse;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.Database;
+import software.amazon.awssdk.services.glue.model.DatabaseInput;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.GetDatabasesRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabasesResponse;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseResponse;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Utilities for Glue catalog Database related operations. */
+@Internal
+public class GlueDatabaseOperator extends GlueOperator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueDatabaseOperator.class);
+
+    public GlueDatabaseOperator(String catalogName, GlueClient glueClient, String glueCatalogId) {
+        super(catalogName, glueClient, glueCatalogId);
+    }
+
+    /**
+     * List all databases present in glue data catalog service.
+     *
+     * @return fully qualified list of database name.
+     */
+    public List<String> listGlueDatabases() throws CatalogException {
+        try {
+            GetDatabasesRequest.Builder databasesRequestBuilder =
+                    GetDatabasesRequest.builder().catalogId(getGlueCatalogId());
+            GetDatabasesResponse response =
+                    glueClient.getDatabases(databasesRequestBuilder.build());
+            GlueUtils.validateGlueResponse(response);
+            List<String> databaseList =
+                    response.databaseList().stream()
+                            .map(Database::name)
+                            .collect(Collectors.toList());
+            String dbResultNextToken = response.nextToken();
+            while (Optional.ofNullable(dbResultNextToken).isPresent()) {
+                databasesRequestBuilder.nextToken(dbResultNextToken);
+                response = glueClient.getDatabases(databasesRequestBuilder.build());
+                GlueUtils.validateGlueResponse(response);
+                databaseList.addAll(
+                        response.databaseList().stream()
+                                .map(Database::name)
+                                .collect(Collectors.toList()));
+                dbResultNextToken = response.nextToken();
+            }
+            return databaseList;
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Create database in glue data catalog service.
+     *
+     * @param databaseName fully qualified name of database.
+     * @param database Instance of {@link CatalogDatabase}.
+     * @throws CatalogException on unexpected error happens.
+     * @throws DatabaseAlreadyExistException when database exists already in glue data catalog.
+     */
+    public void createGlueDatabase(String databaseName, CatalogDatabase database)
+            throws CatalogException, DatabaseAlreadyExistException {
+        GlueUtils.validate(databaseName);
+        Map<String, String> properties = new HashMap<>(database.getProperties());
+        DatabaseInput databaseInput =
+                DatabaseInput.builder()
+                        .name(databaseName)
+                        .description(database.getComment())
+                        .parameters(properties)
+                        .build();
+        CreateDatabaseRequest.Builder requestBuilder =
+                CreateDatabaseRequest.builder()
+                        .databaseInput(databaseInput)
+                        .catalogId(getGlueCatalogId());
+        try {
+            CreateDatabaseResponse response = glueClient.createDatabase(requestBuilder.build());
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+        } catch (EntityNotFoundException e) {
+            throw new CatalogException(catalogName, e);
+        } catch (AlreadyExistsException e) {
+            throw new DatabaseAlreadyExistException(catalogName, databaseName, e);
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Delete a database from Glue data catalog service only when database is empty.
+     *
+     * @param databaseName fully qualified name of database.
+     * @throws CatalogException on unexpected error happens.
+     * @throws DatabaseNotExistException when database doesn't exists in glue catalog.
+     */
+    public void dropGlueDatabase(String databaseName)
+            throws CatalogException, DatabaseNotExistException {
+        GlueUtils.validate(databaseName);
+        DeleteDatabaseRequest deleteDatabaseRequest =
+                DeleteDatabaseRequest.builder()
+                        .name(databaseName)
+                        .catalogId(getGlueCatalogId())
+                        .build();
+        try {
+            DeleteDatabaseResponse response = glueClient.deleteDatabase(deleteDatabaseRequest);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+        } catch (EntityNotFoundException e) {
+            throw new DatabaseNotExistException(catalogName, databaseName);
+        } catch (GlueException e) {
+            throw new CatalogException(catalogName, e);
+        }
+    }
+
+    /**
+     * Delete list of table in database from glue data catalog service.
+     *
+     * @param databaseName fully qualified name of database.
+     * @param tables List of table to remove from database.
+     * @throws CatalogException on unexpected Exception thrown.
+     */
+    public void deleteTablesFromDatabase(String databaseName, Collection<String> tables)
+            throws CatalogException {
+        GlueUtils.validate(databaseName);
+        BatchDeleteTableRequest batchTableRequest =
+                BatchDeleteTableRequest.builder()
+                        .databaseName(databaseName)
+                        .catalogId(getGlueCatalogId())
+                        .tablesToDelete(tables)
+                        .build();
+        try {
+            BatchDeleteTableResponse response = glueClient.batchDeleteTable(batchTableRequest);
+            if (response.hasErrors()) {
+                String errorMsg =
+                        String.format(
+                                "Glue Table errors:- %s",
+                                response.errors().stream()
+                                        .map(
+                                                e ->
+                                                        "Table: "
+                                                                + e.tableName()
+                                                                + "\tErrorDetail: "
+                                                                + e.errorDetail().errorMessage())
+                                        .collect(Collectors.joining("\n")));
+                LOG.error(errorMsg);
+                throw new CatalogException(errorMsg);
+            }
+            GlueUtils.validateGlueResponse(response);
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Delete list of user defined function associated with Database from glue data catalog service.
+     *
+     * @param databaseName fully qualified name of database.
+     * @param functions List of functions to remove from database.
+     * @throws CatalogException on unexpected Exception thrown.
+     */
+    public void deleteFunctionsFromDatabase(String databaseName, Collection<String> functions)
+            throws CatalogException {
+        GlueUtils.validate(databaseName);
+        DeleteUserDefinedFunctionRequest.Builder requestBuilder =
+                DeleteUserDefinedFunctionRequest.builder()
+                        .databaseName(databaseName)
+                        .catalogId(getGlueCatalogId());
+        DeleteUserDefinedFunctionResponse response;
+        for (String functionName : functions) {
+            requestBuilder.functionName(functionName);
+            try {
+                response = glueClient.deleteUserDefinedFunction(requestBuilder.build());
+            } catch (GlueException e) {
+                LOG.error(
+                        "Error deleting function {} in database: {}\n{}",
+                        functionName,
+                        databaseName,
+                        e);
+                throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+            }
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+        }
+    }
+
+    /**
+     * Get {@link CatalogDatabase} instance using the information from glue data-catalog service.
+     *
+     * @param databaseName fully qualified name of database.
+     * @return Instance of {@link CatalogDatabase } .
+     * @throws DatabaseNotExistException when database doesn't exists in Glue data catalog Service.
+     * @throws CatalogException when any unknown error occurs in Execution.
+     */
+    public CatalogDatabase getDatabase(String databaseName)
+            throws DatabaseNotExistException, CatalogException {
+        GlueUtils.validate(databaseName);
+        GetDatabaseRequest getDatabaseRequest =
+                GetDatabaseRequest.builder()
+                        .name(databaseName)
+                        .catalogId(getGlueCatalogId())
+                        .build();
+        try {
+            GetDatabaseResponse response = glueClient.getDatabase(getDatabaseRequest);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+            return GlueUtils.getCatalogDatabase(response.database());
+        } catch (EntityNotFoundException e) {
+            throw new DatabaseNotExistException(catalogName, databaseName);
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Update Database in Glue Metastore.
+     *
+     * @param databaseName Database name.
+     * @param newDatabase instance of {@link CatalogDatabase}.
+     * @throws CatalogException in case of Errors.
+     */
+    public void updateGlueDatabase(String databaseName, CatalogDatabase newDatabase)
+            throws CatalogException {
+        GlueUtils.validate(databaseName);
+        Map<String, String> newProperties = new HashMap<>(newDatabase.getProperties());
+        DatabaseInput databaseInput =
+                DatabaseInput.builder()
+                        .parameters(newProperties)
+                        .description(newDatabase.getComment())
+                        .name(databaseName)
+                        .build();
+
+        UpdateDatabaseRequest updateRequest =
+                UpdateDatabaseRequest.builder()
+                        .databaseInput(databaseInput)
+                        .name(databaseName)
+                        .catalogId(getGlueCatalogId())
+                        .build();
+        UpdateDatabaseResponse response = glueClient.updateDatabase(updateRequest);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug(GlueUtils.getDebugLog(response));
+        }
+        GlueUtils.validateGlueResponse(response);
+        LOG.info("Updated Database: {}", databaseName);
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java
new file mode 100644
index 00000000..8fa32143
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueFunctionOperator.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.operator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogFunctionImpl;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.util.GlueUtils;
+import org.apache.flink.table.resource.ResourceUri;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.AlreadyExistsException;
+import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.PrincipalType;
+import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunction;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunctionInput;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+/** Utilities for Glue catalog Function related operations. */
+@Internal
+public class GlueFunctionOperator extends GlueOperator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueFunctionOperator.class);
+
+    public GlueFunctionOperator(String catalogName, GlueClient glueClient, String glueCatalogId) {
+        super(catalogName, glueClient, glueCatalogId);
+    }
+
+    /**
+     * Create a function. Function name should be handled in a case-insensitive way.
+     *
+     * @param functionPath path of the function
+     * @param function Flink function to be created
+     * @throws CatalogException in case of any runtime exception
+     */
+    public void createGlueFunction(ObjectPath functionPath, CatalogFunction function)
+            throws CatalogException, FunctionAlreadyExistException {
+        UserDefinedFunctionInput functionInput = createFunctionInput(functionPath, function);
+        CreateUserDefinedFunctionRequest.Builder createUDFRequest =
+                CreateUserDefinedFunctionRequest.builder()
+                        .databaseName(functionPath.getDatabaseName())
+                        .catalogId(getGlueCatalogId())
+                        .functionInput(functionInput);
+        try {
+            CreateUserDefinedFunctionResponse response =
+                    glueClient.createUserDefinedFunction(createUDFRequest.build());
+            GlueUtils.validateGlueResponse(response);
+            LOG.info("Created Function: {}", functionPath.getFullName());
+        } catch (AlreadyExistsException e) {
+            LOG.error(
+                    String.format(
+                            "%s already Exists. Function language of type: %s. \n%s",
+                            functionPath.getFullName(), function.getFunctionLanguage(), e));
+            throw new FunctionAlreadyExistException(catalogName, functionPath, e);
+        } catch (GlueException e) {
+            LOG.error("Error creating glue function: {}\n{}", functionPath.getFullName(), e);
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Get the user defined function from glue Catalog. Function name should be handled in a
+     * case-insensitive way.
+     *
+     * @param functionPath path of the function
+     * @return the requested function
+     * @throws CatalogException in case of any runtime exception
+     */
+    public CatalogFunction getGlueFunction(ObjectPath functionPath) {
+        GetUserDefinedFunctionRequest request =
+                GetUserDefinedFunctionRequest.builder()
+                        .databaseName(functionPath.getDatabaseName())
+                        .catalogId(getGlueCatalogId())
+                        .functionName(functionPath.getObjectName())
+                        .build();
+        GetUserDefinedFunctionResponse response = glueClient.getUserDefinedFunction(request);
+        GlueUtils.validateGlueResponse(response);
+        UserDefinedFunction udf = response.userDefinedFunction();
+        List<ResourceUri> resourceUris =
+                udf.resourceUris().stream()
+                        .map(
+                                resourceUri ->
+                                        new org.apache.flink.table.resource.ResourceUri(
+                                                org.apache.flink.table.resource.ResourceType
+                                                        .valueOf(resourceUri.resourceType().name()),
+                                                resourceUri.uri()))
+                        .collect(Collectors.toList());
+        return new CatalogFunctionImpl(
+                GlueUtils.getCatalogFunctionClassName(udf),
+                GlueUtils.getFunctionalLanguage(udf),
+                resourceUris);
+    }
+
+    public List<String> listGlueFunctions(String databaseName) {
+        GetUserDefinedFunctionsRequest.Builder functionsRequest =
+                GetUserDefinedFunctionsRequest.builder()
+                        .databaseName(databaseName)
+                        .catalogId(getGlueCatalogId());
+        List<String> glueFunctions;
+        try {
+            GetUserDefinedFunctionsResponse functionsResponse =
+                    glueClient.getUserDefinedFunctions(functionsRequest.build());
+            String token = functionsResponse.nextToken();
+            glueFunctions =
+                    functionsResponse.userDefinedFunctions().stream()
+                            .map(UserDefinedFunction::functionName)
+                            .collect(Collectors.toCollection(LinkedList::new));
+            while (Optional.ofNullable(token).isPresent()) {
+                functionsRequest.nextToken(token);
+                functionsResponse = glueClient.getUserDefinedFunctions(functionsRequest.build());
+                glueFunctions.addAll(
+                        functionsResponse.userDefinedFunctions().stream()
+                                .map(UserDefinedFunction::functionName)
+                                .collect(Collectors.toCollection(LinkedList::new)));
+                token = functionsResponse.nextToken();
+            }
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+        return glueFunctions;
+    }
+
+    public boolean glueFunctionExists(ObjectPath functionPath) {
+        GetUserDefinedFunctionRequest request =
+                GetUserDefinedFunctionRequest.builder()
+                        .functionName(functionPath.getObjectName())
+                        .databaseName(functionPath.getDatabaseName())
+                        .catalogId(getGlueCatalogId())
+                        .build();
+
+        try {
+            GetUserDefinedFunctionResponse response = glueClient.getUserDefinedFunction(request);
+            GlueUtils.validateGlueResponse(response);
+            return response.userDefinedFunction() != null;
+        } catch (EntityNotFoundException e) {
+            return false;
+        } catch (GlueException e) {
+            LOG.error(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Modify an existing function. Function name should be handled in a case-insensitive way.
+     *
+     * @param functionPath path of function.
+     * @param newFunction modified function.
+     * @throws CatalogException on runtime errors.
+     */
+    public void alterGlueFunction(ObjectPath functionPath, CatalogFunction newFunction)
+            throws CatalogException {
+
+        UserDefinedFunctionInput functionInput = createFunctionInput(functionPath, newFunction);
+
+        UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest =
+                UpdateUserDefinedFunctionRequest.builder()
+                        .functionName(functionPath.getObjectName())
+                        .databaseName(functionPath.getDatabaseName())
+                        .catalogId(getGlueCatalogId())
+                        .functionInput(functionInput)
+                        .build();
+        UpdateUserDefinedFunctionResponse response =
+                glueClient.updateUserDefinedFunction(updateUserDefinedFunctionRequest);
+        GlueUtils.validateGlueResponse(response);
+        LOG.info("Altered Function: {}", functionPath.getFullName());
+    }
+
+    /**
+     * Drop / Delete UserDefinedFunction from glue data catalog.
+     *
+     * @param functionPath fully qualified function path
+     * @throws CatalogException In case of Unexpected errors.
+     */
+    public void dropGlueFunction(ObjectPath functionPath) throws CatalogException {
+        DeleteUserDefinedFunctionRequest request =
+                DeleteUserDefinedFunctionRequest.builder()
+                        .catalogId(getGlueCatalogId())
+                        .functionName(functionPath.getObjectName())
+                        .databaseName(functionPath.getDatabaseName())
+                        .build();
+        DeleteUserDefinedFunctionResponse response = glueClient.deleteUserDefinedFunction(request);
+        GlueUtils.validateGlueResponse(response);
+        LOG.info("Dropped Function: {}", functionPath.getFullName());
+    }
+
+    /**
+     * Utility method to Create UserDefinedFunctionInput instance.
+     *
+     * @param functionPath fully qualified for function path.
+     * @param function Catalog Function instance.
+     * @return User defined function input instance for Glue.
+     * @throws UnsupportedOperationException in case of unsupported operation encountered.
+     */
+    public static UserDefinedFunctionInput createFunctionInput(
+            final ObjectPath functionPath, final CatalogFunction function)
+            throws UnsupportedOperationException {
+        Collection<software.amazon.awssdk.services.glue.model.ResourceUri> resourceUris =
+                new LinkedList<>();
+        for (org.apache.flink.table.resource.ResourceUri resourceUri :
+                function.getFunctionResources()) {
+            switch (resourceUri.getResourceType()) {
+                case JAR:
+                case FILE:
+                case ARCHIVE:
+                    resourceUris.add(
+                            software.amazon.awssdk.services.glue.model.ResourceUri.builder()
+                                    .resourceType(resourceUri.getResourceType().name())
+                                    .uri(resourceUri.getUri())
+                                    .build());
+                    break;
+                default:
+                    throw new UnsupportedOperationException(
+                            "GlueCatalog supports only creating resources JAR/FILE or ARCHIVE.");
+            }
+        }
+        return UserDefinedFunctionInput.builder()
+                .functionName(functionPath.getObjectName())
+                .className(GlueUtils.getGlueFunctionClassName(function))
+                .ownerType(PrincipalType.USER)
+                .ownerName(GlueCatalogConstants.FLINK_CATALOG)
+                .resourceUris(resourceUris)
+                .build();
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java
new file mode 100644
index 00000000..be77913d
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueOperator.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.operator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.StringUtils;
+
+import software.amazon.awssdk.services.glue.GlueClient;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Glue related operation. Important Note : * <a
+ * href="https://aws.amazon.com/premiumsupport/knowledge-center/glue-crawler-internal-service-exception/">...</a>
+ */
+@Internal
+public abstract class GlueOperator {
+
+    public final String glueCatalogId;
+
+    protected final GlueClient glueClient;
+
+    public final String catalogName;
+
+    public GlueOperator(String catalogName, GlueClient glueClient, String glueCatalogId) {
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(catalogName),
+                "catalogName name cannot be null or empty.");
+        checkNotNull(glueClient, "GlueClient Instance cannot be Null.");
+        checkArgument(
+                !StringUtils.isNullOrWhitespaceOnly(glueCatalogId),
+                "glue Catalog Id name cannot be null or empty.");
+        this.catalogName = catalogName;
+        this.glueClient = glueClient;
+        this.glueCatalogId = glueCatalogId;
+    }
+
+    public String getGlueCatalogId() {
+        return glueCatalogId;
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java
new file mode 100644
index 00000000..e612c3fa
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GluePartitionOperator.java
@@ -0,0 +1,484 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.operator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.PartitionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.util.GlueUtils;
+import org.apache.flink.table.expressions.Expression;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.Column;
+import software.amazon.awssdk.services.glue.model.CreatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.CreatePartitionResponse;
+import software.amazon.awssdk.services.glue.model.DeletePartitionRequest;
+import software.amazon.awssdk.services.glue.model.DeletePartitionResponse;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetPartitionRequest;
+import software.amazon.awssdk.services.glue.model.GetPartitionResponse;
+import software.amazon.awssdk.services.glue.model.GetPartitionsRequest;
+import software.amazon.awssdk.services.glue.model.GetPartitionsResponse;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.Partition;
+import software.amazon.awssdk.services.glue.model.PartitionInput;
+import software.amazon.awssdk.services.glue.model.StorageDescriptor;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.UpdatePartitionResponse;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/** Utilities for Glue catalog Partition related operations. */
+@Internal
+public class GluePartitionOperator extends GlueOperator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GluePartitionOperator.class);
+
+    public GluePartitionOperator(String catalogName, GlueClient glueClient, String glueCatalogId) {
+        super(catalogName, glueClient, glueCatalogId);
+    }
+
+    /**
+     * Create partition in glue data catalog service.
+     *
+     * @param glueTable glue table
+     * @param partitionSpec partition spec
+     * @param catalogPartition partition to add.
+     * @throws CatalogException when partition is unable to be created.
+     * @throws PartitionSpecInvalidException when partition specification is invalid.
+     */
+    public void createGluePartition(
+            final Table glueTable,
+            final CatalogPartitionSpec partitionSpec,
+            final CatalogPartition catalogPartition)
+            throws CatalogException, PartitionSpecInvalidException {
+        Map<String, String> catalogPartitionProperties =
+                new HashMap<>(catalogPartition.getProperties());
+        String comment = catalogPartition.getComment();
+        Map<String, String> partitionSpecProperties =
+                new HashMap<>(partitionSpec.getPartitionSpec());
+        if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                    "Partition Keys retrieved from glue table: {}",
+                    glueTable.partitionKeys().stream()
+                            .map(Column::name)
+                            .collect(Collectors.toList()));
+        }
+        List<String> partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys());
+        List<String> partitionValues =
+                getOrderedFullPartitionValues(
+                        partitionSpec,
+                        partitionColumns,
+                        new ObjectPath(glueTable.databaseName(), glueTable.name()));
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Partition Values are: {}", String.join(", ", partitionValues));
+        }
+        for (int i = 0; i < partitionColumns.size(); i++) {
+            if (isNullOrWhitespaceOnly(partitionValues.get(i))) {
+                throw new PartitionSpecInvalidException(
+                        catalogName,
+                        partitionColumns,
+                        new ObjectPath(glueTable.databaseName(), glueTable.name()),
+                        partitionSpec);
+            }
+        }
+        StorageDescriptor.Builder storageDescriptor = glueTable.storageDescriptor().toBuilder();
+        storageDescriptor.parameters(partitionSpecProperties);
+        catalogPartitionProperties.put(GlueCatalogConstants.COMMENT, comment);
+        PartitionInput.Builder partitionInput =
+                PartitionInput.builder()
+                        .parameters(catalogPartitionProperties)
+                        .lastAccessTime(Instant.now())
+                        .storageDescriptor(storageDescriptor.build())
+                        .values(partitionValues);
+        CreatePartitionRequest createPartitionRequest =
+                CreatePartitionRequest.builder()
+                        .partitionInput(partitionInput.build())
+                        .catalogId(getGlueCatalogId())
+                        .databaseName(glueTable.databaseName())
+                        .tableName(glueTable.name())
+                        .build();
+        try {
+            CreatePartitionResponse response = glueClient.createPartition(createPartitionRequest);
+            GlueUtils.validateGlueResponse(response);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Alter Partition in glue data catalog service.
+     *
+     * @param tablePath contains database name and table name.
+     * @param partitionSpec Existing partition information.
+     * @param newPartition Partition information with new changes.
+     * @throws CatalogException Exception in failure.
+     */
+    public void alterGluePartition(
+            final ObjectPath tablePath,
+            final Table glueTable,
+            final CatalogPartitionSpec partitionSpec,
+            final CatalogPartition newPartition)
+            throws CatalogException, PartitionSpecInvalidException {
+        Map<String, String> partitionSpecProperties =
+                new HashMap<>(partitionSpec.getPartitionSpec());
+        Map<String, String> newPartitionProperties = new HashMap<>(newPartition.getProperties());
+        String comment = newPartition.getComment();
+        List<String> partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys());
+        List<String> partitionValues =
+                getOrderedFullPartitionValues(
+                        partitionSpec,
+                        partitionColumns,
+                        new ObjectPath(glueTable.databaseName(), glueTable.name()));
+        StorageDescriptor.Builder storageDescriptor = glueTable.storageDescriptor().toBuilder();
+        storageDescriptor.parameters(partitionSpecProperties);
+        newPartitionProperties.put(GlueCatalogConstants.COMMENT, comment);
+        PartitionInput.Builder partitionInput =
+                PartitionInput.builder()
+                        .lastAccessTime(Instant.now())
+                        .parameters(newPartitionProperties)
+                        .storageDescriptor(storageDescriptor.build())
+                        .values(partitionValues);
+        UpdatePartitionRequest.Builder updatePartitionRequest =
+                UpdatePartitionRequest.builder()
+                        .partitionInput(partitionInput.build())
+                        .databaseName(tablePath.getDatabaseName())
+                        .catalogId(getGlueCatalogId())
+                        .tableName(tablePath.getObjectName())
+                        .partitionValueList(partitionValues);
+        UpdatePartitionResponse response =
+                glueClient.updatePartition(updatePartitionRequest.build());
+        GlueUtils.validateGlueResponse(response);
+    }
+
+    /**
+     * Drop partition from Glue data catalog service.
+     *
+     * @param tablePath fully qualified table path
+     * @param partitionSpec partition spec details
+     * @throws CatalogException in case of unknown errors
+     */
+    public void dropGluePartition(
+            final ObjectPath tablePath,
+            final CatalogPartitionSpec partitionSpec,
+            final Table glueTable)
+            throws CatalogException {
+        try {
+            List<String> partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys());
+            DeletePartitionRequest deletePartitionRequest =
+                    DeletePartitionRequest.builder()
+                            .catalogId(getGlueCatalogId())
+                            .databaseName(tablePath.getDatabaseName())
+                            .tableName(tablePath.getObjectName())
+                            .partitionValues(
+                                    getOrderedFullPartitionValues(
+                                            partitionSpec, partitionColumns, tablePath))
+                            .build();
+            DeletePartitionResponse response = glueClient.deletePartition(deletePartitionRequest);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+        } catch (PartitionSpecInvalidException e) {
+            throw new CatalogException("Invalid Partition Spec", e);
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Retrieve partition from glue data catalog service.
+     *
+     * @param glueTable Instance of {@link Table} from glue data Catalog.
+     * @param partitionSpec instance of {@link CatalogPartitionSpec} containing details of partition
+     * @return Instance of {@link Partition} matching the given partitionSpec.
+     * @throws PartitionNotExistException when partition doesn't exist in Glue data catalog.
+     */
+    public Partition getGluePartition(
+            final Table glueTable, final CatalogPartitionSpec partitionSpec)
+            throws PartitionNotExistException {
+        ObjectPath tablePath = new ObjectPath(glueTable.databaseName(), glueTable.name());
+        try {
+            List<String> partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys());
+            List<String> partitionValues =
+                    getOrderedFullPartitionValues(partitionSpec, partitionColumns, tablePath);
+            LOG.info("Partition values are: {}", String.join(", ", partitionValues));
+            GetPartitionRequest request =
+                    GetPartitionRequest.builder()
+                            .catalogId(getGlueCatalogId())
+                            .databaseName(glueTable.databaseName())
+                            .tableName(glueTable.name())
+                            .partitionValues(partitionValues)
+                            .build();
+            GetPartitionResponse response = glueClient.getPartition(request);
+            GlueUtils.validateGlueResponse(response);
+            Partition partition = response.partition();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(
+                        "(catalogPartition properties) Partition Parameters: {}",
+                        partition.parameters().entrySet().stream()
+                                .map(e -> e.getKey() + " - " + e.getValue())
+                                .collect(Collectors.joining(", ")));
+                LOG.debug(
+                        "(PartitionSpec properties) Partition Parameters: {}",
+                        partition.storageDescriptor().parameters().entrySet().stream()
+                                .map(e -> e.getKey() + " - " + e.getValue())
+                                .collect(Collectors.joining(", ")));
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            if (partition.hasValues()) {
+                return partition;
+            }
+        } catch (EntityNotFoundException e) {
+            throw new PartitionNotExistException(catalogName, tablePath, partitionSpec);
+        } catch (PartitionSpecInvalidException e) {
+            throw new CatalogException("PartitionSpec Invalid ", e);
+        }
+        return null;
+    }
+
+    /**
+     * check Partition exists in glue data catalog.
+     *
+     * @param tablePath Fully Qualified tablePath.
+     * @param partitionSpec Instance of {@link CatalogPartitionSpec}.
+     * @return weather partition exists ?
+     * @throws CatalogException in case of unknown errors.
+     */
+    public boolean gluePartitionExists(
+            final ObjectPath tablePath,
+            final Table glueTable,
+            final CatalogPartitionSpec partitionSpec)
+            throws CatalogException {
+        try {
+            List<String> partitionColumns = GlueUtils.getColumnNames(glueTable.partitionKeys());
+            List<String> partitionValues =
+                    getOrderedFullPartitionValues(partitionSpec, partitionColumns, tablePath);
+            GetPartitionRequest request =
+                    GetPartitionRequest.builder()
+                            .catalogId(getGlueCatalogId())
+                            .databaseName(tablePath.getDatabaseName())
+                            .tableName(tablePath.getObjectName())
+                            .partitionValues(partitionValues)
+                            .build();
+            GetPartitionResponse response = glueClient.getPartition(request);
+            GlueUtils.validateGlueResponse(response);
+            return response.partition()
+                    .storageDescriptor()
+                    .parameters()
+                    .keySet()
+                    .containsAll(partitionSpec.getPartitionSpec().keySet());
+        } catch (EntityNotFoundException e) {
+            LOG.warn(String.format("%s is not found", partitionSpec.getPartitionSpec()));
+        } catch (GlueException e) {
+            throw new CatalogException(catalogName, e);
+        } catch (PartitionSpecInvalidException e) {
+            throw new CatalogException("PartitionSpec Invalid ", e);
+        }
+        return false;
+    }
+
+    /**
+     * Get List of CatalogPartitionSpec from glue data catalog associated with Table.
+     *
+     * @param tablePath fully qualified table path.
+     * @return List of PartitionSpec
+     */
+    public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath) {
+
+        GetPartitionsRequest.Builder getPartitionRequest =
+                GetPartitionsRequest.builder()
+                        .catalogId(getGlueCatalogId())
+                        .databaseName(tablePath.getDatabaseName())
+                        .tableName(tablePath.getObjectName());
+        try {
+            GetPartitionsResponse response = glueClient.getPartitions(getPartitionRequest.build());
+            GlueUtils.validateGlueResponse(response);
+            List<CatalogPartitionSpec> finalPartitionsList =
+                    response.partitions().stream()
+                            .map(this::getCatalogPartitionSpec)
+                            .collect(Collectors.toList());
+            String partitionsResultNextToken = response.nextToken();
+            while (Optional.ofNullable(partitionsResultNextToken).isPresent()) {
+                getPartitionRequest.nextToken(partitionsResultNextToken);
+                response = glueClient.getPartitions(getPartitionRequest.build());
+                finalPartitionsList.addAll(
+                        response.partitions().stream()
+                                .map(this::getCatalogPartitionSpec)
+                                .collect(Collectors.toList()));
+                partitionsResultNextToken = response.nextToken();
+            }
+            return finalPartitionsList;
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Get list of CatalogPartitionSpec from glue data catalog service associated with table and
+     * matches given CatalogPartitionSpec.
+     *
+     * @param tablePath Fully qualified table Path.
+     * @param partitionSpec Partition spec .
+     * @return List of CatalogPartitionSpec.
+     */
+    public List<CatalogPartitionSpec> listPartitions(
+            ObjectPath tablePath, CatalogPartitionSpec partitionSpec) {
+        List<CatalogPartitionSpec> partitionSpecList = listPartitions(tablePath);
+        return partitionSpecList.stream()
+                .filter(
+                        currPartSpec ->
+                                currPartSpec
+                                        .getPartitionSpec()
+                                        .entrySet()
+                                        .containsAll(partitionSpec.getPartitionSpec().entrySet()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Get List of CatalogPartitionSpec of partition by expression filters from glue data catalog
+     * associated with the given Table.
+     *
+     * @param tablePath Fully Qualified Table Path.
+     * @param filters List of Filters.
+     * @return List of Partition Spec
+     */
+    public List<CatalogPartitionSpec> listGluePartitionsByFilter(
+            ObjectPath tablePath, List<Expression> filters) {
+        String expression =
+                filters.stream()
+                        .map(GlueUtils::getExpressionString)
+                        .collect(
+                                Collectors.joining(
+                                        GlueCatalogConstants.SPACE
+                                                + GlueCatalogConstants.AND
+                                                + GlueCatalogConstants.SPACE));
+        try {
+            GetPartitionsRequest.Builder getPartitionsRequest =
+                    GetPartitionsRequest.builder()
+                            .databaseName(tablePath.getDatabaseName())
+                            .tableName(tablePath.getObjectName())
+                            .catalogId(getGlueCatalogId())
+                            .expression(expression);
+            GetPartitionsResponse response = glueClient.getPartitions(getPartitionsRequest.build());
+            List<CatalogPartitionSpec> catalogPartitionSpecList =
+                    response.partitions().stream()
+                            .map(this::getCatalogPartitionSpec)
+                            .collect(Collectors.toList());
+            GlueUtils.validateGlueResponse(response);
+            String nextToken = response.nextToken();
+            while (Optional.ofNullable(nextToken).isPresent()) {
+                getPartitionsRequest.nextToken(nextToken);
+                response = glueClient.getPartitions(getPartitionsRequest.build());
+                GlueUtils.validateGlueResponse(response);
+                catalogPartitionSpecList.addAll(
+                        response.partitions().stream()
+                                .map(this::getCatalogPartitionSpec)
+                                .collect(Collectors.toList()));
+                nextToken = response.nextToken();
+            }
+            return catalogPartitionSpecList;
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Get a list of ordered partition values by re-arranging them based on the given list of
+     * partition keys. If the partition value is null, it'll be converted into default partition
+     * name.
+     *
+     * @param partitionSpec a partition spec.
+     * @param partitionKeys a list of partition keys.
+     * @param tablePath path of the table to which the partition belongs.
+     * @return A list of partition values ordered according to partitionKeys.
+     * @throws PartitionSpecInvalidException thrown if partitionSpec and partitionKeys have
+     *     different sizes, or any key in partitionKeys doesn't exist in partitionSpec.
+     */
+    private List<String> getOrderedFullPartitionValues(
+            CatalogPartitionSpec partitionSpec, List<String> partitionKeys, ObjectPath tablePath)
+            throws PartitionSpecInvalidException {
+        Map<String, String> spec = partitionSpec.getPartitionSpec();
+        if (spec.size() != partitionKeys.size()) {
+            throw new PartitionSpecInvalidException(
+                    catalogName, partitionKeys, tablePath, partitionSpec);
+        }
+
+        List<String> values = new ArrayList<>(spec.size());
+        for (String key : partitionKeys) {
+            if (!spec.containsKey(key)) {
+                throw new PartitionSpecInvalidException(
+                        catalogName, partitionKeys, tablePath, partitionSpec);
+            } else {
+                String value = spec.get(key);
+                if (value == null) {
+                    value = GlueCatalogConstants.DEFAULT_PARTITION_NAME;
+                }
+                values.add(value);
+            }
+        }
+
+        return values;
+    }
+
+    /**
+     * Validate and ensure Table is Partitioned.
+     *
+     * @param tablePath Fully Qualified TablePath.
+     * @param glueTable Instance of {@link Table} from glue data catalog.
+     * @throws TableNotPartitionedException In case of table is not partitioned.
+     */
+    public void ensurePartitionedTable(ObjectPath tablePath, Table glueTable)
+            throws TableNotPartitionedException {
+        if (!glueTable.hasPartitionKeys()) {
+            throw new TableNotPartitionedException(catalogName, tablePath);
+        }
+    }
+
+    /**
+     * Derive {@link CatalogPartitionSpec} instance from {@link Partition}.
+     *
+     * @param partition Glue Partition instance
+     * @return {@link CatalogPartitionSpec} instance
+     */
+    private CatalogPartitionSpec getCatalogPartitionSpec(Partition partition) {
+        Map<String, String> params = new HashMap<>(partition.storageDescriptor().parameters());
+        return new CatalogPartitionSpec(params);
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java
new file mode 100644
index 00000000..ac1877d0
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/operator/GlueTableOperator.java
@@ -0,0 +1,366 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.operator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogView;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.util.GlueUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.model.Column;
+import software.amazon.awssdk.services.glue.model.CreateTableRequest;
+import software.amazon.awssdk.services.glue.model.CreateTableResponse;
+import software.amazon.awssdk.services.glue.model.DeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.DeleteTableResponse;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetTableRequest;
+import software.amazon.awssdk.services.glue.model.GetTableResponse;
+import software.amazon.awssdk.services.glue.model.GetTablesRequest;
+import software.amazon.awssdk.services.glue.model.GetTablesResponse;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.StorageDescriptor;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.TableInput;
+import software.amazon.awssdk.services.glue.model.UpdateTableRequest;
+import software.amazon.awssdk.services.glue.model.UpdateTableResponse;
+
+import java.time.Instant;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Utilities for Glue Table related operations. */
+@Internal
+public class GlueTableOperator extends GlueOperator {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueTableOperator.class);
+
+    public GlueTableOperator(String catalogName, GlueClient glueClient, String glueCatalogId) {
+        super(catalogName, glueClient, glueCatalogId);
+    }
+
+    /**
+     * Create table in glue data catalog service.
+     *
+     * @param tablePath Fully qualified name of table. {@link ObjectPath}
+     * @param table instance of {@link CatalogBaseTable} containing table related information.
+     * @throws CatalogException on unexpected error happens.
+     */
+    public void createGlueTable(final ObjectPath tablePath, final CatalogBaseTable table)
+            throws CatalogException {
+
+        checkNotNull(tablePath, "tablePath cannot be null");
+        checkNotNull(table, "table cannot be null");
+        checkArgument(table instanceof ResolvedCatalogBaseTable, "table should be resolved");
+
+        final Map<String, String> tableProperties = new HashMap<>(table.getOptions());
+        String tableOwner = GlueUtils.extractTableOwner(tableProperties);
+        List<Column> glueTableColumns = GlueUtils.getGlueColumnsFromCatalogTable(table);
+        StorageDescriptor.Builder storageDescriptorBuilder =
+                StorageDescriptor.builder()
+                        .inputFormat(GlueUtils.extractInputFormat(tableProperties))
+                        .outputFormat(GlueUtils.extractOutputFormat(tableProperties));
+
+        TableInput.Builder tableInputBuilder =
+                TableInput.builder()
+                        .name(tablePath.getObjectName())
+                        .description(table.getComment())
+                        .tableType(table.getTableKind().name())
+                        .lastAccessTime(Instant.now())
+                        .owner(tableOwner)
+                        .viewExpandedText(GlueUtils.getExpandedQuery(table))
+                        .viewOriginalText(GlueUtils.getOriginalQuery(table));
+
+        CreateTableRequest.Builder requestBuilder =
+                CreateTableRequest.builder()
+                        .catalogId(getGlueCatalogId())
+                        .databaseName(tablePath.getDatabaseName());
+
+        if (table instanceof CatalogTable) {
+            CatalogTable catalogTable = (CatalogTable) table;
+            if (catalogTable.isPartitioned()) {
+                LOG.info("table is partitioned");
+                Collection<Column> partitionKeys =
+                        GlueUtils.getPartitionKeys(catalogTable, glueTableColumns);
+                tableInputBuilder.partitionKeys(partitionKeys);
+            }
+        }
+
+        try {
+            storageDescriptorBuilder.columns(glueTableColumns);
+            tableInputBuilder.storageDescriptor(storageDescriptorBuilder.build());
+            tableInputBuilder.parameters(tableProperties);
+            requestBuilder.tableInput(tableInputBuilder.build());
+            CreateTableResponse response = glueClient.createTable(requestBuilder.build());
+            GlueUtils.validateGlueResponse(response);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            LOG.info("Created Table: {}", tablePath.getFullName());
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Update Table in glue data catalog service.
+     *
+     * @param tablePath fully Qualified Table Path.
+     * @param newTable instance of {@link CatalogBaseTable} containing information for table.
+     * @throws CatalogException Glue related exception.
+     */
+    public void alterGlueTable(ObjectPath tablePath, CatalogBaseTable newTable)
+            throws CatalogException {
+
+        Map<String, String> tableProperties = new HashMap<>(newTable.getOptions());
+        String tableOwner = GlueUtils.extractTableOwner(tableProperties);
+        List<Column> glueColumns = GlueUtils.getGlueColumnsFromCatalogTable(newTable);
+
+        StorageDescriptor.Builder storageDescriptorBuilder =
+                StorageDescriptor.builder()
+                        .inputFormat(GlueUtils.extractInputFormat(tableProperties))
+                        .outputFormat(GlueUtils.extractOutputFormat(tableProperties))
+                        .parameters(tableProperties)
+                        .columns(glueColumns);
+
+        TableInput.Builder tableInputBuilder =
+                TableInput.builder()
+                        .name(tablePath.getObjectName())
+                        .description(newTable.getComment())
+                        .tableType(newTable.getTableKind().name())
+                        .lastAccessTime(Instant.now())
+                        .owner(tableOwner);
+
+        UpdateTableRequest.Builder requestBuilder =
+                UpdateTableRequest.builder()
+                        .tableInput(tableInputBuilder.build())
+                        .catalogId(getGlueCatalogId())
+                        .databaseName(tablePath.getDatabaseName());
+
+        if (newTable instanceof CatalogTable) {
+            CatalogTable catalogTable = (CatalogTable) newTable;
+            if (catalogTable.isPartitioned()) {
+                tableInputBuilder.partitionKeys(
+                        GlueUtils.getPartitionKeys(catalogTable, glueColumns));
+            }
+        }
+
+        tableInputBuilder.storageDescriptor(storageDescriptorBuilder.build());
+        requestBuilder.tableInput(tableInputBuilder.build());
+
+        try {
+            UpdateTableResponse response = glueClient.updateTable(requestBuilder.build());
+            if (LOG.isDebugEnabled()) {
+                LOG.debug(GlueUtils.getDebugLog(response));
+            }
+            GlueUtils.validateGlueResponse(response);
+            LOG.info("Updated Table: {}", tablePath.getFullName());
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Get List of name of table/view in database based on type identifier. An empty list is
+     * returned if database doesn't contain any table/view.
+     *
+     * @param databaseName fully qualified database name.
+     * @param type type identifier.
+     * @return a list of table/view name in database based on type identifier.
+     * @throws CatalogException in case of any runtime exception.
+     */
+    public List<String> getGlueTableList(String databaseName, String type) throws CatalogException {
+        GetTablesRequest.Builder tablesRequestBuilder =
+                GetTablesRequest.builder().databaseName(databaseName).catalogId(getGlueCatalogId());
+        GetTablesResponse response = glueClient.getTables(tablesRequestBuilder.build());
+        GlueUtils.validateGlueResponse(response);
+        List<String> finalTableList =
+                response.tableList().stream()
+                        .filter(table -> table.tableType().equalsIgnoreCase(type))
+                        .map(Table::name)
+                        .collect(Collectors.toList());
+        String tableResultNextToken = response.nextToken();
+        while (Optional.ofNullable(tableResultNextToken).isPresent()) {
+            tablesRequestBuilder.nextToken(tableResultNextToken);
+            response = glueClient.getTables(tablesRequestBuilder.build());
+            GlueUtils.validateGlueResponse(response);
+            finalTableList.addAll(
+                    response.tableList().stream()
+                            .filter(table -> table.tableType().equalsIgnoreCase(type))
+                            .map(Table::name)
+                            .collect(Collectors.toList()));
+            tableResultNextToken = response.nextToken();
+        }
+        return finalTableList;
+    }
+
+    /**
+     * Returns {@link Table} instance identified by the given {@link ObjectPath}.
+     *
+     * @param tablePath Path of the table or view.
+     * @return The requested table. Glue encapsulates whether table or view in its attribute called
+     *     type.
+     * @throws TableNotExistException if the target does not exist
+     * @throws CatalogException in case of any runtime exception
+     */
+    public Table getGlueTable(ObjectPath tablePath)
+            throws TableNotExistException, CatalogException {
+
+        checkNotNull(tablePath, "TablePath cannot be Null");
+
+        GetTableRequest tablesRequest =
+                GetTableRequest.builder()
+                        .databaseName(tablePath.getDatabaseName())
+                        .name(tablePath.getObjectName())
+                        .catalogId(getGlueCatalogId())
+                        .build();
+        try {
+            GetTableResponse response = glueClient.getTable(tablesRequest);
+            GlueUtils.validateGlueResponse(response);
+            return response.table();
+        } catch (EntityNotFoundException e) {
+            throw new TableNotExistException(catalogName, tablePath, e);
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Check if a table or view exists in glue data catalog service.
+     *
+     * @param tablePath Path of the table or view
+     * @return true if the given table exists in the catalog false otherwise
+     * @throws CatalogException in case of any runtime exception
+     */
+    public boolean glueTableExists(ObjectPath tablePath) throws CatalogException {
+        try {
+            Table glueTable = getGlueTable(tablePath);
+            return glueTable != null && glueTable.name().equals(tablePath.getObjectName());
+        } catch (TableNotExistException e) {
+            return false;
+        } catch (CatalogException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Drop table/view from glue data catalog service.
+     *
+     * @param tablePath fully qualified Table Path.
+     * @throws CatalogException on runtime errors.
+     */
+    public void dropGlueTable(ObjectPath tablePath) throws CatalogException {
+        DeleteTableRequest.Builder tableRequestBuilder =
+                DeleteTableRequest.builder()
+                        .databaseName(tablePath.getDatabaseName())
+                        .name(tablePath.getObjectName())
+                        .catalogId(getGlueCatalogId());
+        try {
+            DeleteTableResponse response = glueClient.deleteTable(tableRequestBuilder.build());
+            GlueUtils.validateGlueResponse(response);
+            LOG.info("Dropped Table: {}", tablePath.getObjectName());
+        } catch (GlueException e) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER, e);
+        }
+    }
+
+    /**
+     * Create {@link CatalogTable} instance from {@link Table} instance.
+     *
+     * @param glueTable Instance of Table from glue Data catalog.
+     * @return {@link CatalogTable} instance.
+     */
+    public CatalogBaseTable getCatalogBaseTableFromGlueTable(Table glueTable) {
+
+        checkNotNull(glueTable, "Glue Table cannot be null");
+        Schema schemaInfo = GlueUtils.getSchemaFromGlueTable(glueTable);
+        List<String> partitionKeys =
+                glueTable.partitionKeys().stream().map(Column::name).collect(Collectors.toList());
+        Map<String, String> properties = new HashMap<>(glueTable.parameters());
+
+        if (glueTable.owner() != null) {
+            properties.put(GlueCatalogConstants.TABLE_OWNER, glueTable.owner());
+        }
+
+        if (glueTable.storageDescriptor().hasParameters()) {
+            properties.putAll(glueTable.storageDescriptor().parameters());
+        }
+
+        if (glueTable.storageDescriptor().inputFormat() != null) {
+            properties.put(
+                    GlueCatalogConstants.TABLE_INPUT_FORMAT,
+                    glueTable.storageDescriptor().inputFormat());
+        }
+
+        if (glueTable.storageDescriptor().outputFormat() != null) {
+            properties.put(
+                    GlueCatalogConstants.TABLE_OUTPUT_FORMAT,
+                    glueTable.storageDescriptor().outputFormat());
+        }
+
+        if (glueTable.tableType().equals(CatalogBaseTable.TableKind.TABLE.name())) {
+            return CatalogTable.of(schemaInfo, glueTable.description(), partitionKeys, properties);
+        } else if (glueTable.tableType().equals(CatalogBaseTable.TableKind.VIEW.name())) {
+            return CatalogView.of(
+                    schemaInfo,
+                    glueTable.description(),
+                    glueTable.viewOriginalText(),
+                    glueTable.viewExpandedText(),
+                    properties);
+
+        } else {
+            throw new CatalogException(
+                    String.format(
+                            "Unknown TableType: %s from Glue Catalog.", glueTable.tableType()));
+        }
+    }
+
+    /**
+     * Glue doesn't Support renaming of table by default. Rename glue table. Glue catalog don't
+     * support renaming table. For renaming in Flink, it has to be done in 3 step. 1. fetch existing
+     * table info from glue 2. Create a table with new-name and use properties of existing table 3.
+     * Delete existing table Note: This above steps are not Atomic in nature.
+     *
+     * <p>Associated issue :- <a href="https://issues.apache.org/jira/browse/FLINK-31926">...</a>
+     *
+     * @param oldTablePath old table name
+     * @param newTablePath new renamed table
+     */
+    public void renameGlueTable(ObjectPath oldTablePath, ObjectPath newTablePath)
+            throws CatalogException, TableNotExistException {
+        throw new UnsupportedOperationException(
+                "Rename Table Operation in Glue Data Catalog is not Supported.");
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java
new file mode 100644
index 00000000..587d7535
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtils.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import org.apache.flink.connector.aws.table.util.AWSOptionUtils;
+import org.apache.flink.connector.aws.table.util.HttpClientOptionUtils;
+import org.apache.flink.connector.base.table.options.ConfigurationValidator;
+import org.apache.flink.connector.base.table.options.TableOptionsUtils;
+import org.apache.flink.table.catalog.glue.GlueCatalogOptions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/** Option Handler for Glue Catalog. */
+@Internal
+public class GlueCatalogOptionsUtils implements TableOptionsUtils, ConfigurationValidator {
+
+    /** Allowed Http Client Types. */
+    private static final String[] ALLOWED_GLUE_HTTP_CLIENTS =
+            new String[] {
+                AWSConfigConstants.CLIENT_TYPE_URLCONNECTION, AWSConfigConstants.CLIENT_TYPE_APACHE
+            };
+
+    private final AWSOptionUtils awsOptionUtils;
+    private final HttpClientOptionUtils httpClientOptionUtils;
+    private final ReadableConfig tableConfig;
+
+    public GlueCatalogOptionsUtils(
+            Map<String, String> resolvedOptions, ReadableConfig tableConfig) {
+        this.awsOptionUtils = new AWSOptionUtils(resolvedOptions);
+        this.httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, resolvedOptions);
+        this.tableConfig = tableConfig;
+    }
+
+    @Override
+    public Properties getValidatedConfigurations() {
+        Properties validatedConfigs = new Properties();
+        validatedConfigs.putAll(awsOptionUtils.getValidatedConfigurations());
+        validatedConfigs.putAll(httpClientOptionUtils.getValidatedConfigurations());
+
+        for (ConfigOption<?> option : GlueCatalogOptions.getAllConfigOptions()) {
+            if (tableConfig.getOptional(option).isPresent()) {
+                validatedConfigs.put(option.key(), tableConfig.getOptional(option).get());
+            }
+        }
+        return validatedConfigs;
+    }
+
+    @Override
+    public Map<String, String> getProcessedResolvedOptions() {
+        Map<String, String> processedOptions = awsOptionUtils.getProcessedResolvedOptions();
+        processedOptions.putAll(httpClientOptionUtils.getProcessedResolvedOptions());
+        return processedOptions;
+    }
+
+    @Override
+    public List<String> getNonValidatedPrefixes() {
+        // Glue Specific Options are handled by FactoryHelper
+        return Arrays.asList(
+                AWSOptionUtils.AWS_PROPERTIES_PREFIX, HttpClientOptionUtils.CLIENT_PREFIX);
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java
new file mode 100644
index 00000000..073ba748
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/java/org/apache/flink/table/catalog/glue/util/GlueUtils.java
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.FunctionLanguage;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogBaseTable;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.glue.GlueCatalogOptions;
+import org.apache.flink.table.catalog.glue.TypeMapper;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.types.AbstractDataType;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.glue.model.Column;
+import software.amazon.awssdk.services.glue.model.Database;
+import software.amazon.awssdk.services.glue.model.GlueResponse;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunction;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.lang3.BooleanUtils.FALSE;
+import static org.apache.commons.lang3.BooleanUtils.TRUE;
+import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.EXPLAIN_EXTRAS;
+import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.IS_PERSISTED;
+import static org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants.IS_PHYSICAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly;
+
+/** Utilities related glue Operation. */
+@Internal
+public class GlueUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(GlueUtils.class);
+
+    /**
+     * Glue supports lowercase naming convention.
+     *
+     * @param name fully qualified name.
+     * @return modified name according to glue convention.
+     */
+    public static String getGlueConventionalName(String name) {
+        return name.toLowerCase(Locale.ROOT);
+    }
+
+    /**
+     * Extract database location from properties and remove location from properties. fallback to
+     * create default location if not present
+     *
+     * @param databaseProperties database properties.
+     * @param databaseName fully qualified name for database.
+     * @param catalogPath catalog path.
+     * @return location for database.
+     */
+    public static String extractDatabaseLocation(
+            final Map<String, String> databaseProperties,
+            final String databaseName,
+            final String catalogPath) {
+        if (databaseProperties.containsKey(GlueCatalogConstants.LOCATION_URI)) {
+            return databaseProperties.remove(GlueCatalogConstants.LOCATION_URI);
+        } else {
+            LOG.info("No location URI Set. Using Catalog Path as default");
+            return catalogPath + GlueCatalogConstants.LOCATION_SEPARATOR + databaseName;
+        }
+    }
+
+    /**
+     * Extract table location from table properties and remove location from properties. fallback to
+     * create default location if not present
+     *
+     * @param tableProperties table properties.
+     * @param tablePath fully qualified object for table.
+     * @param catalogPath catalog path.
+     * @return location for table.
+     */
+    public static String extractTableLocation(
+            final Map<String, String> tableProperties,
+            final ObjectPath tablePath,
+            final String catalogPath) {
+        if (tableProperties.containsKey(GlueCatalogConstants.LOCATION_URI)) {
+            return tableProperties.remove(GlueCatalogConstants.LOCATION_URI);
+        } else {
+            return catalogPath
+                    + GlueCatalogConstants.LOCATION_SEPARATOR
+                    + tablePath.getDatabaseName()
+                    + GlueCatalogConstants.LOCATION_SEPARATOR
+                    + tablePath.getObjectName();
+        }
+    }
+
+    /**
+     * Build CatalogDatabase instance using information from glue Database instance.
+     *
+     * @param glueDatabase {@link Database }
+     * @return {@link CatalogDatabase } instance.
+     */
+    public static CatalogDatabase getCatalogDatabase(final Database glueDatabase) {
+        Map<String, String> properties = new HashMap<>(glueDatabase.parameters());
+        return new CatalogDatabaseImpl(properties, glueDatabase.description());
+    }
+
+    /**
+     * A Glue database name cannot be longer than 255 characters. The only acceptable characters are
+     * lowercase letters, numbers, and the underscore character. More details: <a
+     * href="https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html">...</a>
+     *
+     * @param name name
+     */
+    public static void validate(String name) {
+        checkArgument(
+                name != null && name.matches(GlueCatalogConstants.GLUE_DB_PATTERN.pattern()),
+                "Database name does not comply with the Glue naming convention. "
+                        + "Check here https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html");
+    }
+
+    /** validate response from client call. */
+    public static void validateGlueResponse(GlueResponse response) {
+        if (response != null && !response.sdkHttpResponse().isSuccessful()) {
+            throw new CatalogException(GlueCatalogConstants.GLUE_EXCEPTION_MSG_IDENTIFIER);
+        }
+    }
+
+    /**
+     * @param udf Instance of UserDefinedFunction
+     * @return ClassName for function
+     */
+    public static String getCatalogFunctionClassName(final UserDefinedFunction udf) {
+        validateUDFClassName(udf.className());
+        String[] splitName = udf.className().split(GlueCatalogConstants.DEFAULT_SEPARATOR);
+        return splitName[splitName.length - 1];
+    }
+
+    /**
+     * Validates UDF class name from glue.
+     *
+     * @param name name of UDF.
+     */
+    private static void validateUDFClassName(final String name) {
+        checkArgument(!isNullOrWhitespaceOnly(name));
+
+        if (name.split(GlueCatalogConstants.DEFAULT_SEPARATOR).length
+                != GlueCatalogConstants.UDF_CLASS_NAME_SIZE) {
+            throw new ValidationException("Improper ClassName: " + name);
+        }
+    }
+
+    /**
+     * Derive functionalLanguage from glue function name. Glue doesn't have any attribute to save
+     * the functionalLanguage Name. Thus, storing FunctionalLanguage in the name itself.
+     *
+     * @param glueFunction Function name from glue.
+     * @return Identifier for FunctionalLanguage.
+     */
+    public static FunctionLanguage getFunctionalLanguage(final UserDefinedFunction glueFunction) {
+        if (glueFunction.className().startsWith(GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX)) {
+            return FunctionLanguage.JAVA;
+        } else if (glueFunction
+                .className()
+                .startsWith(GlueCatalogConstants.FLINK_PYTHON_FUNCTION_PREFIX)) {
+            return FunctionLanguage.PYTHON;
+        } else if (glueFunction
+                .className()
+                .startsWith(GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX)) {
+            return FunctionLanguage.SCALA;
+        } else {
+            throw new CatalogException(
+                    "Invalid Functional Language for className: " + glueFunction.className());
+        }
+    }
+
+    /**
+     * Get expanded Query from CatalogBaseTable.
+     *
+     * @param table Instance of catalogBaseTable.
+     * @return expandedQuery for Glue Table.
+     */
+    public static String getExpandedQuery(CatalogBaseTable table) {
+        // https://issues.apache.org/jira/browse/FLINK-31961
+        return "";
+    }
+
+    /**
+     * Get Original Query from CatalogBaseTable.
+     *
+     * @param table Instance of CatalogBaseTable.
+     * @return OriginalQuery for Glue Table.
+     */
+    public static String getOriginalQuery(CatalogBaseTable table) {
+        // https://issues.apache.org/jira/browse/FLINK-31961
+        return "";
+    }
+
+    /**
+     * Extract table owner name and remove from properties.
+     *
+     * @param properties Map of properties.
+     * @return fully qualified owner name.
+     */
+    public static String extractTableOwner(Map<String, String> properties) {
+        return properties.containsKey(GlueCatalogConstants.TABLE_OWNER)
+                ? properties.remove(GlueCatalogConstants.TABLE_OWNER)
+                : null;
+    }
+
+    /**
+     * Derive Instance of Glue Column from {@link CatalogBaseTable}.
+     *
+     * @param flinkColumn Instance of {@link org.apache.flink.table.catalog.Column}.
+     * @throws CatalogException Throws exception in case of failure.
+     */
+    public static Column getGlueColumn(org.apache.flink.table.catalog.Column flinkColumn)
+            throws CatalogException {
+        return Column.builder()
+                .comment(flinkColumn.asSummaryString())
+                .type(TypeMapper.mapFlinkTypeToGlueType(flinkColumn.getDataType().getLogicalType()))
+                .name(flinkColumn.getName())
+                .parameters(buildGlueColumnParams(flinkColumn))
+                .build();
+    }
+
+    public static Map<String, String> buildGlueColumnParams(
+            org.apache.flink.table.catalog.Column column) {
+        Map<String, String> params = new HashMap<>();
+        params.put(IS_PERSISTED, column.isPersisted() ? TRUE : FALSE);
+        params.put(EXPLAIN_EXTRAS, column.explainExtras().orElse(null));
+        params.put(IS_PHYSICAL, column.isPhysical() ? TRUE : FALSE);
+        return params;
+    }
+
+    /**
+     * Build set of {@link Column} associated with table.
+     *
+     * @param table instance of {@link CatalogBaseTable}.
+     * @return List of Column
+     */
+    public static List<Column> getGlueColumnsFromCatalogTable(final CatalogBaseTable table) {
+        ResolvedCatalogBaseTable<?> resolvedTable = (ResolvedCatalogBaseTable<?>) table;
+        return resolvedTable.getResolvedSchema().getColumns().stream()
+                .map(GlueUtils::getGlueColumn)
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Extract InputFormat from properties if present and remove inputFormat from properties.
+     * fallback to default format if not present
+     *
+     * @param tableProperties Key/Value properties
+     * @return input Format.
+     */
+    public static String extractInputFormat(final Map<String, String> tableProperties) {
+        return tableProperties.containsKey(GlueCatalogConstants.TABLE_INPUT_FORMAT)
+                ? tableProperties.remove(GlueCatalogConstants.TABLE_INPUT_FORMAT)
+                : GlueCatalogOptions.INPUT_FORMAT.defaultValue();
+    }
+
+    /**
+     * Extract OutputFormat from properties if present and remove outputFormat from properties.
+     * fallback to default format if not present
+     *
+     * @param tableProperties Key/Value properties
+     * @return output Format.
+     */
+    public static String extractOutputFormat(Map<String, String> tableProperties) {
+        return tableProperties.containsKey(GlueCatalogConstants.TABLE_OUTPUT_FORMAT)
+                ? tableProperties.remove(GlueCatalogConstants.TABLE_OUTPUT_FORMAT)
+                : GlueCatalogOptions.OUTPUT_FORMAT.defaultValue();
+    }
+
+    /**
+     * Get list of filtered columns which are partition columns.
+     *
+     * @param catalogTable {@link CatalogTable} instance.
+     * @param columns List of all column in table.
+     * @return List of column marked as partition key.
+     */
+    public static Collection<Column> getPartitionKeys(
+            CatalogTable catalogTable, Collection<Column> columns) {
+        Set<String> partitionKeys = new HashSet<>(catalogTable.getPartitionKeys());
+        return columns.stream()
+                .filter(column -> partitionKeys.contains(column.name()))
+                .collect(Collectors.toList());
+    }
+
+    public static String getDebugLog(final GlueResponse response) {
+        return String.format(
+                "Glue response : status = %s \n" + "Details = %s \nMetadataResponse = %s",
+                response.sdkHttpResponse().isSuccessful(),
+                response.sdkHttpResponse().toString(),
+                response.responseMetadata());
+    }
+
+    /**
+     * Derive {@link Schema} from Glue {@link Table}.
+     *
+     * @param glueTable Instance of {@link Table}
+     * @return {@link Schema} of table.
+     */
+    public static Schema getSchemaFromGlueTable(Table glueTable) {
+        List<Column> columns = glueTable.storageDescriptor().columns();
+        Schema.Builder schemaBuilder = Schema.newBuilder();
+        for (Column column : columns) {
+            String columnName = column.name();
+            String columnType = column.type().toLowerCase();
+            AbstractDataType<?> flinkDataType = TypeMapper.glueTypeToFlinkType(columnType);
+            schemaBuilder.column(columnName, flinkDataType);
+        }
+        return schemaBuilder.build();
+    }
+
+    /**
+     * Get column names from List of {@link Column}.
+     *
+     * @param columns List of {@link Column}.
+     * @return Names of all Columns.
+     */
+    public static List<String> getColumnNames(final List<Column> columns) {
+        return columns.stream().map(Column::name).collect(Collectors.toList());
+    }
+
+    /**
+     * Function ClassName pattern to be kept in Glue Data Catalog.
+     *
+     * @param function Catalog Function.
+     * @return function class name.
+     */
+    public static String getGlueFunctionClassName(CatalogFunction function) {
+        switch (function.getFunctionLanguage()) {
+            case JAVA:
+                return GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX + function.getClassName();
+            case SCALA:
+                return GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX + function.getClassName();
+            case PYTHON:
+                return GlueCatalogConstants.FLINK_PYTHON_FUNCTION_PREFIX + function.getClassName();
+            default:
+                throw new UnsupportedOperationException(
+                        "GlueCatalog supports only creating: "
+                                + Arrays.stream(FunctionLanguage.values())
+                                        .map(FunctionLanguage::name)
+                                        .collect(
+                                                Collectors.joining(
+                                                        GlueCatalogConstants.NEXT_LINE)));
+        }
+    }
+
+    /**
+     * Derive the expression string from given {@link Expression}.
+     *
+     * @param expression Instance of {@link Expression}.
+     * @return Derived String from {@link Expression}.
+     */
+    public static String getExpressionString(Expression expression) {
+        return getExpressionString(expression, new StringBuilder());
+    }
+
+    /**
+     * Recursively derive the expression string from given {@link Expression}.
+     *
+     * @param expression Instance of {@link Expression}.
+     * @param sb Used to build the derived expression string during recursion.
+     * @return Derived String from {@link Expression}.
+     */
+    private static String getExpressionString(Expression expression, StringBuilder sb) {
+        for (Expression childExpression : expression.getChildren()) {
+            if (childExpression.getChildren() != null && !childExpression.getChildren().isEmpty()) {
+                getExpressionString(childExpression, sb);
+            }
+        }
+
+        // If the StringBuilder is not empty, append "AND "
+        if (sb.length() > 0) {
+            sb.append(GlueCatalogConstants.SPACE)
+                    .append(GlueCatalogConstants.AND)
+                    .append(GlueCatalogConstants.SPACE);
+        }
+
+        // Append the current expression summary
+        sb.append(expression.asSummaryString());
+
+        return sb.toString();
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE
new file mode 100644
index 00000000..3f91d1a2
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,7 @@
+flink-catalog-aws-glue
+Copyright 2014-2023 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
new file mode 100644
index 00000000..d7d504eb
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+org.apache.flink.table.catalog.glue.factory.GlueCatalogFactory
\ No newline at end of file
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java
new file mode 100644
index 00000000..0be1eea6
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/DummyGlueClient.java
@@ -0,0 +1,623 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import lombok.Data;
+import software.amazon.awssdk.awscore.exception.AwsServiceException;
+import software.amazon.awssdk.core.exception.SdkClientException;
+import software.amazon.awssdk.services.glue.GlueClient;
+import software.amazon.awssdk.services.glue.GlueServiceClientConfiguration;
+import software.amazon.awssdk.services.glue.model.AlreadyExistsException;
+import software.amazon.awssdk.services.glue.model.BatchDeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.BatchDeleteTableResponse;
+import software.amazon.awssdk.services.glue.model.ConcurrentModificationException;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.CreatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.CreatePartitionResponse;
+import software.amazon.awssdk.services.glue.model.CreateTableRequest;
+import software.amazon.awssdk.services.glue.model.CreateTableResponse;
+import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.Database;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.DeleteDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.DeletePartitionRequest;
+import software.amazon.awssdk.services.glue.model.DeletePartitionResponse;
+import software.amazon.awssdk.services.glue.model.DeleteTableRequest;
+import software.amazon.awssdk.services.glue.model.DeleteTableResponse;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.DeleteUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.EntityNotFoundException;
+import software.amazon.awssdk.services.glue.model.GetDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.GetDatabasesRequest;
+import software.amazon.awssdk.services.glue.model.GetDatabasesResponse;
+import software.amazon.awssdk.services.glue.model.GetPartitionRequest;
+import software.amazon.awssdk.services.glue.model.GetPartitionResponse;
+import software.amazon.awssdk.services.glue.model.GetPartitionsRequest;
+import software.amazon.awssdk.services.glue.model.GetPartitionsResponse;
+import software.amazon.awssdk.services.glue.model.GetTableRequest;
+import software.amazon.awssdk.services.glue.model.GetTableResponse;
+import software.amazon.awssdk.services.glue.model.GetTablesRequest;
+import software.amazon.awssdk.services.glue.model.GetTablesResponse;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsRequest;
+import software.amazon.awssdk.services.glue.model.GetUserDefinedFunctionsResponse;
+import software.amazon.awssdk.services.glue.model.GlueEncryptionException;
+import software.amazon.awssdk.services.glue.model.GlueException;
+import software.amazon.awssdk.services.glue.model.InternalServiceException;
+import software.amazon.awssdk.services.glue.model.InvalidInputException;
+import software.amazon.awssdk.services.glue.model.InvalidStateException;
+import software.amazon.awssdk.services.glue.model.OperationTimeoutException;
+import software.amazon.awssdk.services.glue.model.Partition;
+import software.amazon.awssdk.services.glue.model.ResourceNotReadyException;
+import software.amazon.awssdk.services.glue.model.ResourceNumberLimitExceededException;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseResponse;
+import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.UpdatePartitionResponse;
+import software.amazon.awssdk.services.glue.model.UpdateTableRequest;
+import software.amazon.awssdk.services.glue.model.UpdateTableResponse;
+import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.UpdateUserDefinedFunctionResponse;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunction;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.dummySdkHttpResponse;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getFullyQualifiedName;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getPartitionFromCreatePartitionRequest;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getTableFromCreateTableRequest;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getTableFromUpdateTableRequest;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getUDFFromCreateUserDefinedFunctionRequest;
+
+/** Dummy Glue client for Test. */
+@Data
+public class DummyGlueClient implements GlueClient {
+
+    public Map<String, Database> databaseMap;
+
+    public Map<String, Table> tableMap;
+
+    public Map<String, UserDefinedFunction> userDefinedFunctionMap;
+
+    public Map<String, List<Partition>> partitionMap;
+
+    @Override
+    public UpdateUserDefinedFunctionResponse updateUserDefinedFunction(
+            UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+        String functionName =
+                getFullyQualifiedName(
+                        updateUserDefinedFunctionRequest.databaseName(),
+                        updateUserDefinedFunctionRequest.functionName());
+        if (!databaseMap.containsKey(updateUserDefinedFunctionRequest.databaseName())
+                || !userDefinedFunctionMap.containsKey(functionName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+        UserDefinedFunction udf = userDefinedFunctionMap.get(functionName);
+        UserDefinedFunction updatedUDF =
+                udf.toBuilder()
+                        .catalogId(updateUserDefinedFunctionRequest.catalogId())
+                        .functionName(updateUserDefinedFunctionRequest.functionName())
+                        .databaseName(updateUserDefinedFunctionRequest.databaseName())
+                        .ownerName(updateUserDefinedFunctionRequest.functionInput().ownerName())
+                        .ownerType(updateUserDefinedFunctionRequest.functionInput().ownerType())
+                        .className(updateUserDefinedFunctionRequest.functionInput().className())
+                        .resourceUris(
+                                updateUserDefinedFunctionRequest.functionInput().resourceUris())
+                        .build();
+        userDefinedFunctionMap.put(functionName, updatedUDF);
+        return (UpdateUserDefinedFunctionResponse)
+                UpdateUserDefinedFunctionResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    public DummyGlueClient() {
+        databaseMap = new HashMap<>();
+        tableMap = new HashMap<>();
+        userDefinedFunctionMap = new HashMap<>();
+        partitionMap = new HashMap<>();
+    }
+
+    @Override
+    public String serviceName() {
+        return "Glue";
+    }
+
+    @Override
+    public void close() {}
+
+    @Override
+    public GlueServiceClientConfiguration serviceClientConfiguration() {
+        return null;
+    }
+
+    @Override
+    public CreateDatabaseResponse createDatabase(CreateDatabaseRequest createDatabaseRequest)
+            throws InvalidInputException, AlreadyExistsException,
+                    ResourceNumberLimitExceededException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException,
+                    ConcurrentModificationException, AwsServiceException, SdkClientException,
+                    GlueException {
+        CreateDatabaseResponse.Builder responseBuilder = CreateDatabaseResponse.builder();
+
+        if (databaseMap.containsKey(createDatabaseRequest.databaseInput().name())) {
+            throw AlreadyExistsException.builder().build();
+        }
+        databaseMap.put(
+                createDatabaseRequest.databaseInput().name(),
+                GlueCatalogTestUtils.getDatabaseFromCreateDatabaseRequest(createDatabaseRequest));
+        return (CreateDatabaseResponse)
+                responseBuilder.sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public UpdateDatabaseResponse updateDatabase(UpdateDatabaseRequest updateDatabaseRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException,
+                    ConcurrentModificationException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        if (!databaseMap.containsKey(updateDatabaseRequest.name())) {
+            throw EntityNotFoundException.builder().build();
+        }
+        databaseMap.remove(updateDatabaseRequest.name());
+        databaseMap.put(
+                updateDatabaseRequest.name(),
+                GlueCatalogTestUtils.getDatabaseFromUpdateDatabaseRequest(updateDatabaseRequest));
+
+        return (UpdateDatabaseResponse)
+                UpdateDatabaseResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public GetDatabaseResponse getDatabase(GetDatabaseRequest getDatabaseRequest)
+            throws InvalidInputException, EntityNotFoundException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+
+        GetDatabaseResponse.Builder responseBuilder =
+                (GetDatabaseResponse.Builder)
+                        GetDatabaseResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200));
+
+        if (!databaseMap.containsKey(getDatabaseRequest.name())) {
+            throw EntityNotFoundException.builder().build();
+        }
+        return responseBuilder.database(databaseMap.get(getDatabaseRequest.name())).build();
+    }
+
+    @Override
+    public GetDatabasesResponse getDatabases(GetDatabasesRequest getDatabasesRequest)
+            throws InvalidInputException, InternalServiceException, OperationTimeoutException,
+                    GlueEncryptionException, AwsServiceException, SdkClientException,
+                    GlueException {
+        return (GetDatabasesResponse)
+                GetDatabasesResponse.builder()
+                        .databaseList(databaseMap.values())
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public DeleteDatabaseResponse deleteDatabase(DeleteDatabaseRequest deleteDatabaseRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, ConcurrentModificationException, AwsServiceException,
+                    SdkClientException, GlueException {
+
+        if (databaseMap.containsKey(deleteDatabaseRequest.name())) {
+            databaseMap.remove(deleteDatabaseRequest.name());
+            return (DeleteDatabaseResponse)
+                    DeleteDatabaseResponse.builder()
+                            .sdkHttpResponse(dummySdkHttpResponse(200))
+                            .build();
+        }
+        throw EntityNotFoundException.builder().build();
+    }
+
+    @Override
+    public CreateTableResponse createTable(CreateTableRequest createTableRequest)
+            throws AlreadyExistsException, InvalidInputException, EntityNotFoundException,
+                    ResourceNumberLimitExceededException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException,
+                    ConcurrentModificationException, ResourceNotReadyException, AwsServiceException,
+                    SdkClientException, GlueException {
+
+        Table table = getTableFromCreateTableRequest(createTableRequest);
+        String tableName =
+                getFullyQualifiedName(
+                        createTableRequest.databaseName(), createTableRequest.tableInput().name());
+        if (tableMap.containsKey(tableName)) {
+            throw AlreadyExistsException.builder().build();
+        }
+
+        tableMap.put(tableName, table);
+        return (CreateTableResponse)
+                CreateTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public UpdateTableResponse updateTable(UpdateTableRequest updateTableRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, ConcurrentModificationException,
+                    ResourceNumberLimitExceededException, GlueEncryptionException,
+                    ResourceNotReadyException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        String tableName =
+                getFullyQualifiedName(
+                        updateTableRequest.databaseName(), updateTableRequest.tableInput().name());
+        if (!databaseMap.containsKey(updateTableRequest.databaseName())
+                || !tableMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        tableMap.put(tableName, getTableFromUpdateTableRequest(updateTableRequest));
+        return (UpdateTableResponse)
+                UpdateTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public GetTableResponse getTable(GetTableRequest getTableRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, ResourceNotReadyException,
+                    AwsServiceException, SdkClientException, GlueException {
+
+        String tableName =
+                getFullyQualifiedName(getTableRequest.databaseName(), getTableRequest.name());
+
+        if (!tableMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        Table table = tableMap.get(tableName);
+        return (GetTableResponse)
+                GetTableResponse.builder()
+                        .table(table)
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public GetTablesResponse getTables(GetTablesRequest getTablesRequest)
+            throws EntityNotFoundException, InvalidInputException, OperationTimeoutException,
+                    InternalServiceException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+        String databaseName = getTablesRequest.databaseName();
+
+        if (!databaseMap.containsKey(databaseName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        List<Table> tables =
+                tableMap.entrySet().stream()
+                        .filter(e -> e.getKey().startsWith(databaseName))
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+        return (GetTablesResponse)
+                GetTablesResponse.builder()
+                        .tableList(tables)
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public DeleteTableResponse deleteTable(DeleteTableRequest deleteTableRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, ConcurrentModificationException,
+                    ResourceNotReadyException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        String tableName =
+                getFullyQualifiedName(deleteTableRequest.databaseName(), deleteTableRequest.name());
+        if (!databaseMap.containsKey(deleteTableRequest.databaseName())
+                || !tableMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        tableMap.remove(tableName);
+        return (DeleteTableResponse)
+                DeleteTableResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public BatchDeleteTableResponse batchDeleteTable(
+            BatchDeleteTableRequest batchDeleteTableRequest)
+            throws InvalidInputException, EntityNotFoundException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, ResourceNotReadyException,
+                    AwsServiceException, SdkClientException, GlueException {
+
+        if (!databaseMap.containsKey(batchDeleteTableRequest.databaseName())) {
+            throw EntityNotFoundException.builder().build();
+        }
+        for (Map.Entry<String, Table> entry : tableMap.entrySet()) {
+            if (entry.getKey().startsWith(batchDeleteTableRequest.databaseName())) {
+                tableMap.remove(entry.getKey());
+            }
+        }
+        return (BatchDeleteTableResponse)
+                BatchDeleteTableResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    // -- partition
+    @Override
+    public CreatePartitionResponse createPartition(CreatePartitionRequest createPartitionRequest)
+            throws InvalidInputException, AlreadyExistsException,
+                    ResourceNumberLimitExceededException, InternalServiceException,
+                    EntityNotFoundException, OperationTimeoutException, GlueEncryptionException,
+                    AwsServiceException, SdkClientException, GlueException {
+        Partition partition = getPartitionFromCreatePartitionRequest(createPartitionRequest);
+        String tableName =
+                getFullyQualifiedName(
+                        createPartitionRequest.databaseName(), createPartitionRequest.tableName());
+        List<Partition> partitionList = partitionMap.getOrDefault(tableName, new ArrayList<>());
+        String partValues = String.join(":", partition.values());
+        for (Partition part : partitionList) {
+            if (String.join(":", part.values()).equals(partValues)) {
+                throw AlreadyExistsException.builder().build();
+            }
+        }
+
+        partitionList.add(partition);
+        partitionMap.put(tableName, partitionList);
+        return (CreatePartitionResponse)
+                CreatePartitionResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public UpdatePartitionResponse updatePartition(UpdatePartitionRequest updatePartitionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+
+        String tableName =
+                getFullyQualifiedName(
+                        updatePartitionRequest.databaseName(), updatePartitionRequest.tableName());
+        if (!partitionMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+        List<Partition> partitionList = partitionMap.get(tableName);
+        String values = String.join(":", updatePartitionRequest.partitionInput().values());
+        for (int i = 0; i < partitionList.size(); i++) {
+            if (values.equals(String.join(":", partitionList.get(i).values()))) {
+                partitionList.remove(i);
+            }
+        }
+        partitionList.add(
+                GlueCatalogTestUtils.getPartitionFromUpdatePartitionRequest(
+                        updatePartitionRequest));
+        partitionMap.put(tableName, partitionList);
+        return (UpdatePartitionResponse)
+                UpdatePartitionResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public GetPartitionResponse getPartition(GetPartitionRequest getPartitionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+        String tableName =
+                getFullyQualifiedName(
+                        getPartitionRequest.databaseName(), getPartitionRequest.tableName());
+        if (!partitionMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+        List<Partition> partitionList = partitionMap.get(tableName);
+        String partitionValues = String.join(":", getPartitionRequest.partitionValues());
+        for (Partition partition : partitionList) {
+            if (partitionValues.equals(String.join(":", partition.values()))) {
+                return (GetPartitionResponse)
+                        GetPartitionResponse.builder()
+                                .partition(partition)
+                                .sdkHttpResponse(dummySdkHttpResponse(200))
+                                .build();
+            }
+        }
+        return (GetPartitionResponse)
+                GetPartitionResponse.builder().sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public DeletePartitionResponse deletePartition(DeletePartitionRequest deletePartitionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        String tableName =
+                getFullyQualifiedName(
+                        deletePartitionRequest.databaseName(), deletePartitionRequest.tableName());
+
+        if (!databaseMap.containsKey(deletePartitionRequest.databaseName())
+                || !tableMap.containsKey(tableName)
+                || !partitionMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        List<Partition> partitions = partitionMap.get(tableName);
+        int pos = 0;
+        for (Partition partition : partitions) {
+            if (matchValues(partition.values(), deletePartitionRequest.partitionValues())) {
+                break;
+            }
+            pos++;
+        }
+        if (pos < partitions.size()) {
+            partitions.remove(pos);
+            partitionMap.remove(tableName);
+            partitionMap.put(tableName, partitions);
+        }
+        return (DeletePartitionResponse)
+                DeletePartitionResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    private boolean matchValues(List<String> gluePartValues, List<String> partValues) {
+        Set<String> gluePartitionValueSet = new HashSet<>(gluePartValues);
+        int count = 0;
+        for (String partVal : partValues) {
+            if (gluePartitionValueSet.contains(partVal)) {
+                count++;
+            }
+        }
+
+        return count == partValues.size();
+    }
+
+    @Override
+    public GetPartitionsResponse getPartitions(GetPartitionsRequest getPartitionsRequest)
+            throws EntityNotFoundException, InvalidInputException, OperationTimeoutException,
+                    InternalServiceException, GlueEncryptionException, InvalidStateException,
+                    ResourceNotReadyException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        String tableName =
+                getFullyQualifiedName(
+                        getPartitionsRequest.databaseName(), getPartitionsRequest.tableName());
+        if (!databaseMap.containsKey(getPartitionsRequest.databaseName())
+                || !tableMap.containsKey(tableName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        return (GetPartitionsResponse)
+                GetPartitionsResponse.builder()
+                        .partitions(partitionMap.getOrDefault(tableName, new ArrayList<>()))
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    // -- functions
+    @Override
+    public CreateUserDefinedFunctionResponse createUserDefinedFunction(
+            CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest)
+            throws AlreadyExistsException, InvalidInputException, InternalServiceException,
+                    EntityNotFoundException, OperationTimeoutException,
+                    ResourceNumberLimitExceededException, GlueEncryptionException,
+                    AwsServiceException, SdkClientException, GlueException {
+        if (!databaseMap.containsKey(createUserDefinedFunctionRequest.databaseName())) {
+            throw EntityNotFoundException.builder().build();
+        }
+        String functionName =
+                getFullyQualifiedName(
+                        createUserDefinedFunctionRequest.databaseName(),
+                        createUserDefinedFunctionRequest.functionInput().functionName());
+        if (userDefinedFunctionMap.containsKey(functionName)) {
+            throw AlreadyExistsException.builder().build();
+        }
+        UserDefinedFunction udf =
+                getUDFFromCreateUserDefinedFunctionRequest(createUserDefinedFunctionRequest);
+        userDefinedFunctionMap.put(functionName, udf);
+        return (CreateUserDefinedFunctionResponse)
+                CreateUserDefinedFunctionResponse.builder()
+                        .sdkHttpResponse(dummySdkHttpResponse(200))
+                        .build();
+    }
+
+    @Override
+    public GetUserDefinedFunctionResponse getUserDefinedFunction(
+            GetUserDefinedFunctionRequest getUserDefinedFunctionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+        if (!databaseMap.containsKey(getUserDefinedFunctionRequest.databaseName())) {
+            throw EntityNotFoundException.builder().build();
+        }
+        String functionName =
+                getFullyQualifiedName(
+                        getUserDefinedFunctionRequest.databaseName(),
+                        getUserDefinedFunctionRequest.functionName());
+        GetUserDefinedFunctionResponse.Builder response = GetUserDefinedFunctionResponse.builder();
+        if (userDefinedFunctionMap.containsKey(functionName)) {
+            response.userDefinedFunction(userDefinedFunctionMap.get(functionName));
+        }
+
+        return (GetUserDefinedFunctionResponse)
+                response.sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+
+    @Override
+    public GetUserDefinedFunctionsResponse getUserDefinedFunctions(
+            GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest)
+            throws EntityNotFoundException, InvalidInputException, OperationTimeoutException,
+                    InternalServiceException, GlueEncryptionException, AwsServiceException,
+                    SdkClientException, GlueException {
+
+        GetUserDefinedFunctionsResponse.Builder response =
+                (GetUserDefinedFunctionsResponse.Builder)
+                        GetUserDefinedFunctionsResponse.builder()
+                                .sdkHttpResponse(dummySdkHttpResponse(200));
+        if (!databaseMap.containsKey(getUserDefinedFunctionsRequest.databaseName())) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        List<UserDefinedFunction> udfs =
+                userDefinedFunctionMap.entrySet().stream()
+                        .filter(
+                                e ->
+                                        e.getKey()
+                                                .startsWith(
+                                                        getUserDefinedFunctionsRequest
+                                                                .databaseName()))
+                        .map(Map.Entry::getValue)
+                        .collect(Collectors.toList());
+        return response.userDefinedFunctions(udfs).build();
+    }
+
+    @Override
+    public DeleteUserDefinedFunctionResponse deleteUserDefinedFunction(
+            DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest)
+            throws EntityNotFoundException, InvalidInputException, InternalServiceException,
+                    OperationTimeoutException, AwsServiceException, SdkClientException,
+                    GlueException {
+
+        String functionName =
+                getFullyQualifiedName(
+                        deleteUserDefinedFunctionRequest.databaseName(),
+                        deleteUserDefinedFunctionRequest.functionName());
+
+        if (!databaseMap.containsKey(deleteUserDefinedFunctionRequest.databaseName())
+                || !userDefinedFunctionMap.containsKey(functionName)) {
+            throw EntityNotFoundException.builder().build();
+        }
+
+        DeleteUserDefinedFunctionResponse.Builder response =
+                DeleteUserDefinedFunctionResponse.builder();
+        userDefinedFunctionMap.remove(functionName);
+        return (DeleteUserDefinedFunctionResponse)
+                response.sdkHttpResponse(dummySdkHttpResponse(200)).build();
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java
new file mode 100644
index 00000000..9974c517
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogOptionsTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.configuration.ConfigOption;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.Set;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.CREDENTIAL_PROVIDER;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.DEFAULT_DATABASE;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_ACCOUNT_ID;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ENDPOINT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ID;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.HTTP_CLIENT_TYPE;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.INPUT_FORMAT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.OUTPUT_FORMAT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.REGION;
+
+class GlueCatalogOptionsTest {
+
+    @Test
+    public void testGetAllConfigOptions() {
+        Set<ConfigOption<?>> allConfigOptions = GlueCatalogOptions.getAllConfigOptions();
+        Assertions.assertEquals(9, allConfigOptions.size());
+        Assertions.assertTrue(allConfigOptions.contains(INPUT_FORMAT));
+        Assertions.assertTrue(allConfigOptions.contains(OUTPUT_FORMAT));
+        Assertions.assertTrue(allConfigOptions.contains(GLUE_CATALOG_ENDPOINT));
+        Assertions.assertTrue(allConfigOptions.contains(GLUE_ACCOUNT_ID));
+        Assertions.assertTrue(allConfigOptions.contains(GLUE_CATALOG_ID));
+        Assertions.assertTrue(allConfigOptions.contains(DEFAULT_DATABASE));
+        Assertions.assertTrue(allConfigOptions.contains(HTTP_CLIENT_TYPE));
+        Assertions.assertTrue(allConfigOptions.contains(REGION));
+        Assertions.assertTrue(allConfigOptions.contains(CREDENTIAL_PROVIDER));
+    }
+
+    @Test
+    public void testGetRequiredConfigOptions() {
+        Set<ConfigOption<?>> requiredOptions = GlueCatalogOptions.getRequiredConfigOptions();
+        Assertions.assertEquals(0, requiredOptions.size());
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java
new file mode 100644
index 00000000..2af1ad82
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTest.java
@@ -0,0 +1,998 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogFunctionImpl;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionImpl;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogView;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.FunctionLanguage;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedCatalogView;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException;
+import org.apache.flink.table.catalog.exceptions.PartitionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.catalog.glue.operator.GlueDatabaseOperator;
+import org.apache.flink.table.catalog.glue.operator.GlueFunctionOperator;
+import org.apache.flink.table.catalog.glue.operator.GluePartitionOperator;
+import org.apache.flink.table.catalog.glue.operator.GlueTableOperator;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.types.DataType;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COLUMN_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COLUMN_2;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.COMMENT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_2;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_DESCRIPTION;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.FUNCTION_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_2;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_3;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_4;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_5;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.VIEW_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.VIEW_2;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDatabaseParams;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogDatabase;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogTable;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyCatalogTableWithPartition;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getDummyTableParams;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.getPartitionSpecParams;
+
+class GlueCatalogTest {
+
+    public static final String WAREHOUSE_PATH = "s3://bucket";
+    private static final String CATALOG_NAME = "glue";
+    private static DummyGlueClient glue;
+    private static GlueCatalog glueCatalog;
+
+    @BeforeAll
+    static void setUp() {
+        glue = new DummyGlueClient();
+        String glueCatalogId = "dummy-catalog-Id";
+        GlueDatabaseOperator glueDatabaseOperator =
+                new GlueDatabaseOperator(CATALOG_NAME, glue, glueCatalogId);
+        GlueTableOperator glueTableOperator =
+                new GlueTableOperator(CATALOG_NAME, glue, glueCatalogId);
+        GluePartitionOperator gluePartitionOperator =
+                new GluePartitionOperator(CATALOG_NAME, glue, glueCatalogId);
+        GlueFunctionOperator glueFunctionOperator =
+                new GlueFunctionOperator(CATALOG_NAME, glue, glueCatalogId);
+        glueCatalog =
+                new GlueCatalog(
+                        CATALOG_NAME,
+                        GlueCatalog.DEFAULT_DB,
+                        glue,
+                        glueDatabaseOperator,
+                        glueTableOperator,
+                        gluePartitionOperator,
+                        glueFunctionOperator);
+    }
+
+    @BeforeEach
+    public void clear() {
+        glue.setDatabaseMap(new HashMap<>());
+        glue.setTableMap(new HashMap<>());
+        glue.setPartitionMap(new HashMap<>());
+        glue.setUserDefinedFunctionMap(new HashMap<>());
+    }
+
+    // ------ Database
+    @Test
+    void testCreateDatabase() throws DatabaseNotExistException {
+
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1));
+        CatalogDatabase catalogDatabase =
+                new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false));
+        CatalogDatabase database = glueCatalog.getDatabase(DATABASE_1);
+        Assertions.assertNotNull(database);
+        Assertions.assertNotNull(database.getProperties());
+        Assertions.assertNotNull(database.getComment());
+        Assertions.assertEquals(DATABASE_DESCRIPTION, database.getComment());
+        Assertions.assertThrows(
+                DatabaseAlreadyExistException.class,
+                () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false));
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_2, catalogDatabase, true));
+    }
+
+    @Test
+    void testAlterDatabase() throws DatabaseNotExistException, DatabaseAlreadyExistException {
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1));
+        CatalogDatabase catalogDatabase =
+                new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        CatalogDatabase database = glueCatalog.getDatabase(DATABASE_1);
+        Assertions.assertNotNull(database);
+        Assertions.assertNotNull(database.getProperties());
+        Assertions.assertNotNull(database.getComment());
+        Assertions.assertEquals(DATABASE_DESCRIPTION, database.getComment());
+
+        Assertions.assertThrows(
+                DatabaseNotExistException.class,
+                () -> glueCatalog.alterDatabase(DATABASE_2, database, false));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.alterDatabase(DATABASE_2, database, true));
+
+        Map<String, String> properties = catalogDatabase.getProperties();
+        properties.put("newKey", "val");
+        CatalogDatabase newCatalogDatabase = catalogDatabase.copy(properties);
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.alterDatabase(DATABASE_1, newCatalogDatabase, false));
+        CatalogDatabase database1 = glueCatalog.getDatabase(DATABASE_1);
+        Assertions.assertNotNull(database1);
+        Assertions.assertNotNull(database1.getProperties());
+        Assertions.assertEquals(database1.getProperties(), properties);
+        Assertions.assertNotNull(database1.getComment());
+        Assertions.assertEquals(DATABASE_DESCRIPTION, database1.getComment());
+    }
+
+    @Test
+    void testDatabaseExists()
+            throws DatabaseAlreadyExistException, DatabaseNotEmptyException,
+                    DatabaseNotExistException {
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+        CatalogDatabase catalogDatabase =
+                new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1));
+        glueCatalog.dropDatabase(DATABASE_1, true, true);
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1));
+        glueCatalog.dropDatabase(DATABASE_1, false, false);
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1));
+        glueCatalog.dropDatabase(DATABASE_1, true, false);
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertTrue(glueCatalog.databaseExists(DATABASE_1));
+        glueCatalog.dropDatabase(DATABASE_1, false, true);
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+    }
+
+    @Test
+    void testDropDatabase() throws DatabaseAlreadyExistException {
+
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, false));
+
+        Assertions.assertThrows(
+                DatabaseNotExistException.class,
+                () -> glueCatalog.dropDatabase(DATABASE_2, false, true));
+
+        Assertions.assertThrows(
+                DatabaseNotExistException.class,
+                () -> glueCatalog.dropDatabase(DATABASE_2, false, false));
+
+        CatalogDatabase catalogDatabase =
+                new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, true));
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1));
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, false));
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, true));
+        glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, true, false));
+    }
+
+    @Test
+    void testListDatabases() {
+        Assertions.assertEquals(new ArrayList<>(), glueCatalog.listDatabases());
+        List<String> expectedDatabasesList = Arrays.asList(DATABASE_1, DATABASE_2);
+        CatalogDatabase catalogDatabase =
+                new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_1, catalogDatabase, false));
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_2, catalogDatabase, false));
+        Assertions.assertEquals(expectedDatabasesList, glueCatalog.listDatabases());
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_1, false, false));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropDatabase(DATABASE_2, false, false));
+    }
+
+    @Test
+    void testGetDatabase() throws DatabaseNotExistException {
+
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_1));
+        Assertions.assertFalse(glueCatalog.databaseExists(DATABASE_2));
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.getDatabase(DATABASE_1));
+        createDatabase(DATABASE_1);
+        CatalogDatabase db = glueCatalog.getDatabase(DATABASE_1);
+        Assertions.assertEquals(getDummyCatalogDatabase().getComment(), db.getComment());
+        Assertions.assertEquals(getDatabaseParams(), db.getProperties());
+    }
+
+    @Test
+    void testIsDatabaseEmpty()
+            throws TableAlreadyExistException, DatabaseNotExistException,
+                    FunctionAlreadyExistException {
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_1, getDummyCatalogDatabase(), false));
+        Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1));
+
+        // create a table for the database
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        glueCatalog.createTable(tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false);
+        Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false));
+        Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1));
+
+        // create userDefinedFunctions for the database
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        Assertions.assertDoesNotThrow(
+                () ->
+                        glueCatalog.createFunction(
+                                functionPath,
+                                GlueCatalogTestUtils.getDummyCatalogFunction(),
+                                false));
+        Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false));
+        Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1));
+
+        // both table and userDefinedFunction are present
+        glueCatalog.createTable(tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false);
+        glueCatalog.createFunction(
+                functionPath, GlueCatalogTestUtils.getDummyCatalogFunction(), false);
+        Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false));
+        Assertions.assertFalse(glueCatalog.isDatabaseEmpty(DATABASE_1));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false));
+        Assertions.assertTrue(glueCatalog.isDatabaseEmpty(DATABASE_1));
+    }
+
+    // ------ Table
+    @Test
+    public void testCreateTable() throws TableNotExistException {
+
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        Assertions.assertThrows(
+                DatabaseNotExistException.class,
+                () -> glueCatalog.createTable(tablePath, getDummyCatalogTable(), false));
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(DATABASE_1, getDummyCatalogDatabase(), false));
+        Assertions.assertDoesNotThrow(
+                () ->
+                        glueCatalog.createTable(
+                                tablePath, GlueCatalogTestUtils.getDummyCatalogTable(), false));
+        CatalogBaseTable table = glueCatalog.getTable(tablePath);
+        Assertions.assertEquals(
+                table.getUnresolvedSchema().getColumns().size(),
+                getDummyCatalogTable().getUnresolvedSchema().getColumns().size());
+        Assertions.assertEquals(table.getTableKind(), getDummyCatalogTable().getTableKind());
+        ObjectPath tablePath2 = new ObjectPath(DATABASE_1, TABLE_2);
+        CatalogBaseTable catalogBaseTable = getDummyCatalogTableWithPartition();
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createTable(tablePath2, catalogBaseTable, false));
+        table = glueCatalog.getTable(tablePath2);
+        Assertions.assertEquals(
+                table.getUnresolvedSchema().getColumns().get(0).getName(),
+                catalogBaseTable.getUnresolvedSchema().getColumns().get(0).getName());
+        Assertions.assertEquals(
+                table.getUnresolvedSchema().getColumns().get(1).getName(),
+                catalogBaseTable.getUnresolvedSchema().getColumns().get(1).getName());
+        Assertions.assertEquals(table.getTableKind(), catalogBaseTable.getTableKind());
+    }
+
+    @Test
+    public void testCreateView() throws TableNotExistException {
+        ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_1);
+        Assertions.assertThrows(TableNotExistException.class, () -> glueCatalog.getTable(viewPath));
+        createDatabase(viewPath.getDatabaseName());
+        createView(viewPath);
+        CatalogBaseTable view = glueCatalog.getTable(viewPath);
+        Assertions.assertNotNull(view);
+        Assertions.assertEquals(getDummyTableParams(), view.getOptions());
+        Assertions.assertEquals(CatalogTable.TableKind.VIEW.name(), view.getTableKind().name());
+        ObjectPath tablePath = new ObjectPath(DATABASE_2, TABLE_2);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        CatalogBaseTable table = glueCatalog.getTable(tablePath);
+        Assertions.assertNotNull(table);
+        Assertions.assertEquals(getDummyTableParams(), table.getOptions());
+        Assertions.assertEquals(CatalogTable.TableKind.TABLE.name(), table.getTableKind().name());
+    }
+
+    @Test
+    public void testGetTable() throws TableNotExistException {
+        Assertions.assertThrows(
+                IllegalArgumentException.class,
+                () -> glueCatalog.getTable(new ObjectPath(null, null)));
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        Assertions.assertThrows(
+                TableNotExistException.class, () -> glueCatalog.getTable(tablePath));
+        createDatabase(tablePath.getDatabaseName());
+        Assertions.assertThrows(
+                TableNotExistException.class, () -> glueCatalog.getTable(tablePath));
+        createTable(tablePath);
+        CatalogBaseTable table = glueCatalog.getTable(tablePath);
+        Assertions.assertNotNull(table);
+        Assertions.assertEquals(
+                CatalogBaseTable.TableKind.TABLE.name(), table.getTableKind().name());
+        Assertions.assertEquals(getDummyTableParams(), table.getOptions());
+    }
+
+    @Test
+    public void testGetView() throws TableNotExistException {
+
+        ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_1);
+        createDatabase(viewPath.getDatabaseName());
+        createView(viewPath);
+        CatalogBaseTable view = glueCatalog.getTable(viewPath);
+        Assertions.assertNotNull(view);
+        Assertions.assertEquals(CatalogBaseTable.TableKind.VIEW.name(), view.getTableKind().name());
+        Assertions.assertEquals(getDummyTableParams(), view.getOptions());
+    }
+
+    @Test
+    public void testTableExists() {
+
+        ObjectPath tablePath = new ObjectPath(DATABASE_2, TABLE_1);
+        Assertions.assertThrows(NullPointerException.class, () -> glueCatalog.getTable(null));
+        Assertions.assertThrows(
+                TableNotExistException.class, () -> glueCatalog.getTable(tablePath));
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.getTable(tablePath));
+        Assertions.assertThrows(
+                TableNotExistException.class,
+                () -> glueCatalog.getTable(new ObjectPath(DATABASE_2, TABLE_2)));
+        Assertions.assertThrows(
+                TableNotExistException.class,
+                () -> glueCatalog.getTable(new ObjectPath(DATABASE_1, TABLE_2)));
+        Assertions.assertTrue(glueCatalog.tableExists(tablePath));
+        Assertions.assertFalse(glueCatalog.tableExists(new ObjectPath(DATABASE_1, TABLE_1)));
+    }
+
+    @Test
+    public void testListTables() throws DatabaseNotExistException {
+        createDatabase(DATABASE_1);
+        createTable(new ObjectPath(DATABASE_1, TABLE_1));
+        createTable(new ObjectPath(DATABASE_1, TABLE_2));
+        createTable(new ObjectPath(DATABASE_1, TABLE_3));
+        createTable(new ObjectPath(DATABASE_1, TABLE_4));
+        createTable(new ObjectPath(DATABASE_1, TABLE_5));
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2));
+        Assertions.assertEquals(5, glueCatalog.listTables(DATABASE_1).size());
+        Assertions.assertEquals(
+                Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5),
+                glueCatalog.listTables(DATABASE_1));
+        createView(new ObjectPath(DATABASE_1, VIEW_1));
+        Assertions.assertEquals(6, glueCatalog.listTables(DATABASE_1).size());
+        Assertions.assertEquals(
+                Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1),
+                glueCatalog.listTables(DATABASE_1));
+    }
+
+    @Test
+    public void testListTablesWithCombinationOfDifferentTableKind()
+            throws DatabaseNotExistException {
+        createDatabase(DATABASE_1);
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.listTables(DATABASE_1));
+        createTable(new ObjectPath(DATABASE_1, TABLE_1));
+        createTable(new ObjectPath(DATABASE_1, TABLE_2));
+        createTable(new ObjectPath(DATABASE_1, TABLE_3));
+        createView(new ObjectPath(DATABASE_1, VIEW_2));
+        createTable(new ObjectPath(DATABASE_1, TABLE_4));
+        createTable(new ObjectPath(DATABASE_1, TABLE_5));
+        createView(new ObjectPath(DATABASE_1, VIEW_1));
+        Assertions.assertEquals(7, glueCatalog.listTables(DATABASE_1).size());
+        Assertions.assertEquals(
+                Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1, VIEW_2),
+                glueCatalog.listTables(DATABASE_1));
+    }
+
+    @Test
+    public void testListView() throws DatabaseNotExistException {
+        createDatabase(DATABASE_1);
+        Assertions.assertThrows(
+                DatabaseNotExistException.class, () -> glueCatalog.listTables(DATABASE_2));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.listTables(DATABASE_1));
+        createTable(new ObjectPath(DATABASE_1, TABLE_1));
+        createTable(new ObjectPath(DATABASE_1, TABLE_2));
+        createTable(new ObjectPath(DATABASE_1, TABLE_3));
+        createView(new ObjectPath(DATABASE_1, VIEW_2));
+        createTable(new ObjectPath(DATABASE_1, TABLE_4));
+        createTable(new ObjectPath(DATABASE_1, TABLE_5));
+        createView(new ObjectPath(DATABASE_1, VIEW_1));
+        Assertions.assertEquals(2, glueCatalog.listViews(DATABASE_1).size());
+        Assertions.assertNotSame(
+                Arrays.asList(TABLE_1, TABLE_2, TABLE_3, TABLE_4, TABLE_5, VIEW_1, VIEW_2),
+                glueCatalog.listViews(DATABASE_1));
+        Assertions.assertEquals(Arrays.asList(VIEW_1, VIEW_2), glueCatalog.listViews(DATABASE_1));
+        Assertions.assertNotSame(
+                Arrays.asList(TABLE_1, TABLE_2, VIEW_1, VIEW_2),
+                glueCatalog.listViews(DATABASE_1),
+                "Should not contain any identifier of type table");
+    }
+
+    @Test
+    public void testAlterTable() throws TableNotExistException {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        Assertions.assertThrows(
+                NullPointerException.class, () -> glueCatalog.alterTable(tablePath, null, false));
+        createTable(tablePath);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.getTable(tablePath));
+        CatalogBaseTable table = glueCatalog.getTable(tablePath);
+        Assertions.assertNotNull(table);
+        Assertions.assertEquals(
+                table.getTableKind().name(), CatalogBaseTable.TableKind.TABLE.name());
+        Assertions.assertEquals(table.getOptions(), getDummyTableParams());
+        Assertions.assertNotNull(table.getUnresolvedSchema());
+        Map<String, String> modifiedOptions = table.getOptions();
+        modifiedOptions.put("newKey", "newValue");
+        Schema schema = table.getUnresolvedSchema();
+        Assertions.assertNotNull(schema);
+
+        Schema modifiedSchema =
+                Schema.newBuilder().fromSchema(schema).column("col3", DataTypes.STRING()).build();
+        List<DataType> dataTypes =
+                Arrays.asList(DataTypes.STRING(), DataTypes.STRING(), DataTypes.STRING());
+        ResolvedSchema resolvedSchema =
+                ResolvedSchema.physical(
+                        modifiedSchema.getColumns().stream()
+                                .map(Schema.UnresolvedColumn::getName)
+                                .collect(Collectors.toList()),
+                        dataTypes);
+        ResolvedCatalogTable table1 =
+                new ResolvedCatalogTable(
+                        CatalogTable.of(
+                                modifiedSchema,
+                                "Changed Comment",
+                                new ArrayList<>(),
+                                modifiedOptions),
+                        resolvedSchema);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.alterTable(tablePath, table1, false));
+        CatalogBaseTable retrievedTable = glueCatalog.getTable(tablePath);
+        Assertions.assertEquals(modifiedOptions, retrievedTable.getOptions());
+        Assertions.assertEquals(
+                modifiedSchema.getColumns().size(),
+                retrievedTable.getUnresolvedSchema().getColumns().size());
+    }
+
+    @Test
+    public void testDropTable() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2);
+        ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_2);
+        createDatabase(tablePath.getDatabaseName());
+        Assertions.assertThrows(
+                TableNotExistException.class, () -> glueCatalog.dropTable(tablePath, false));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, true));
+        createTable(tablePath);
+        createView(viewPath);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false));
+        Assertions.assertThrows(
+                TableNotExistException.class, () -> glueCatalog.getTable(tablePath));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, true));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(viewPath, false));
+        Assertions.assertThrows(TableNotExistException.class, () -> glueCatalog.getTable(viewPath));
+    }
+
+    @Test
+    public void testRenameTable() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2);
+        ObjectPath viewPath = new ObjectPath(DATABASE_1, VIEW_2);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        createView(viewPath);
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.renameTable(tablePath, TABLE_4, false));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.renameTable(viewPath, VIEW_1, false));
+    }
+
+    // ------- Function
+    @Test
+    public void testCreateFunction() {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        createDatabase(functionPath.getDatabaseName());
+        Assertions.assertFalse(glueCatalog.functionExists(functionPath));
+        createFunction(functionPath, FunctionLanguage.JAVA, "TestClass");
+        Assertions.assertTrue(glueCatalog.functionExists(functionPath));
+    }
+
+    @Test
+    public void testNormalize() {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, "Function-1");
+        ObjectPath normalizeFunctionPath = glueCatalog.normalize(functionPath);
+        Assertions.assertNotNull(normalizeFunctionPath);
+        Assertions.assertEquals(DATABASE_1, normalizeFunctionPath.getDatabaseName());
+        Assertions.assertEquals("function-1", normalizeFunctionPath.getObjectName());
+    }
+
+    @Test
+    public void testAlterFunction() {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        createDatabase(functionPath.getDatabaseName());
+        Assertions.assertFalse(glueCatalog.functionExists(functionPath));
+        CatalogFunction catalogFunction =
+                new CatalogFunctionImpl("ClassName", FunctionLanguage.JAVA);
+        Assertions.assertThrows(
+                FunctionNotExistException.class,
+                () -> glueCatalog.alterFunction(functionPath, catalogFunction, true));
+
+        createFunction(functionPath, FunctionLanguage.JAVA, "TestClass");
+        Assertions.assertTrue(glueCatalog.functionExists(functionPath));
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.alterFunction(functionPath, catalogFunction, false));
+    }
+
+    @Test
+    public void testDropFunction() {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        createDatabase(functionPath.getDatabaseName());
+        Assertions.assertThrows(
+                FunctionNotExistException.class,
+                () -> glueCatalog.dropFunction(functionPath, false));
+        createFunction(functionPath, FunctionLanguage.JAVA, "TestClass");
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false));
+        Assertions.assertThrows(
+                FunctionNotExistException.class,
+                () -> glueCatalog.dropFunction(functionPath, false));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, true));
+    }
+
+    @Test
+    public void testListFunctions() throws DatabaseNotExistException {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        String className = GlueCatalogConstants.FLINK_SCALA_FUNCTION_PREFIX + "TestClass";
+        createDatabase(DATABASE_1);
+        createFunction(functionPath, FunctionLanguage.SCALA, className);
+        Assertions.assertDoesNotThrow(() -> glueCatalog.listFunctions(DATABASE_1));
+        List<String> udfList = glueCatalog.listFunctions(DATABASE_1);
+        Assertions.assertNotNull(udfList);
+        Assertions.assertEquals(1, udfList.size());
+    }
+
+    @Test
+    public void testGetFunction() throws FunctionNotExistException {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        createDatabase(functionPath.getDatabaseName());
+        String className = GlueCatalogConstants.FLINK_JAVA_FUNCTION_PREFIX + "TestClass";
+        createFunction(functionPath, FunctionLanguage.JAVA, className);
+        Assertions.assertThrows(
+                ValidationException.class, () -> glueCatalog.getFunction(functionPath));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropFunction(functionPath, false));
+        createFunction(functionPath, FunctionLanguage.JAVA, "TestClass");
+        CatalogFunction catalogFunction = glueCatalog.getFunction(functionPath);
+        Assertions.assertNotNull(catalogFunction);
+        Assertions.assertEquals(FunctionLanguage.JAVA, catalogFunction.getFunctionLanguage());
+        Assertions.assertEquals(3, catalogFunction.getFunctionResources().size());
+        Assertions.assertEquals("TestClass", catalogFunction.getClassName());
+    }
+
+    @Test
+    public void testFunctionExists() {
+        ObjectPath functionPath = new ObjectPath(DATABASE_1, FUNCTION_1);
+        Assertions.assertFalse(glueCatalog.functionExists(functionPath));
+        createDatabase(functionPath.getDatabaseName());
+        Assertions.assertFalse(glueCatalog.functionExists(functionPath));
+        createFunction(functionPath, FunctionLanguage.JAVA, "TestClass");
+        Assertions.assertTrue(glueCatalog.functionExists(functionPath));
+    }
+
+    // ------ Partition
+    @Test
+    public void testCreatePartition() throws PartitionNotExistException {
+
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        CatalogPartition catalogPartition =
+                new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT);
+        Assertions.assertDoesNotThrow(
+                () ->
+                        glueCatalog.createPartition(
+                                tablePath, partitionSpec, catalogPartition, false));
+
+        CatalogPartition partition = glueCatalog.getPartition(tablePath, partitionSpec);
+        Assertions.assertNotNull(partition);
+        Assertions.assertEquals(getPartitionSpecParams(), partition.getProperties());
+
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () -> glueCatalog.createPartition(null, partitionSpec, catalogPartition, false));
+
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () -> glueCatalog.createPartition(tablePath, null, catalogPartition, false));
+
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () -> glueCatalog.createPartition(tablePath, partitionSpec, null, false));
+
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        glueCatalog.getPartition(
+                                tablePath, new CatalogPartitionSpec(new HashMap<>())));
+
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () -> glueCatalog.getPartition(tablePath, new CatalogPartitionSpec(null)));
+
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        glueCatalog.createPartition(
+                                tablePath,
+                                new CatalogPartitionSpec(new HashMap<>()),
+                                catalogPartition,
+                                false));
+
+        Assertions.assertThrows(
+                PartitionAlreadyExistsException.class,
+                () ->
+                        glueCatalog.createPartition(
+                                tablePath,
+                                partitionSpec,
+                                new CatalogPartitionImpl(new HashMap<>(), COMMENT),
+                                false));
+    }
+
+    @Test
+    public void testListPartitions()
+            throws TableNotPartitionedException, TableNotExistException,
+                    PartitionSpecInvalidException {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_2);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        Assertions.assertEquals(
+                0,
+                glueCatalog
+                        .listPartitions(tablePath, new CatalogPartitionSpec(new HashMap<>()))
+                        .size());
+        createPartition(tablePath);
+        Assertions.assertEquals(
+                1,
+                glueCatalog
+                        .listPartitions(tablePath, new CatalogPartitionSpec(new HashMap<>()))
+                        .size());
+        Map<String, String> partSpec = new HashMap<>();
+        partSpec.put(COLUMN_1, "v1");
+        partSpec.put(COLUMN_2, "v2");
+        Assertions.assertEquals(
+                new CatalogPartitionSpec(partSpec),
+                glueCatalog
+                        .listPartitions(
+                                tablePath, new CatalogPartitionSpec(getPartitionSpecParams()))
+                        .get(0));
+    }
+
+    @Test
+    public void testIsPartitionedTable() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        createNonPartitionedTable(tablePath);
+        Assertions.assertFalse(glueCatalog.isPartitionedTable(tablePath));
+        Assertions.assertDoesNotThrow(() -> glueCatalog.dropTable(tablePath, false));
+        createTable(tablePath);
+        createPartition(tablePath);
+        Assertions.assertTrue(glueCatalog.isPartitionedTable(tablePath));
+    }
+
+    @Test
+    public void testListPartitionsByFilter()
+            throws TableNotPartitionedException, TableNotExistException {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        createPartition(tablePath);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        Assertions.assertDoesNotThrow(() -> glueCatalog.getPartition(tablePath, partitionSpec));
+        List<Expression> expressions = new ArrayList<>();
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.listPartitionsByFilter(tablePath, expressions));
+        List<CatalogPartitionSpec> partitionSpecs =
+                glueCatalog.listPartitionsByFilter(tablePath, expressions);
+        Assertions.assertNotNull(partitionSpecs);
+        Assertions.assertEquals(1, partitionSpecs.size());
+        Assertions.assertEquals(getPartitionSpecParams(), partitionSpecs.get(0).getPartitionSpec());
+    }
+
+    @Test
+    public void testDropPartition() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        Assertions.assertThrows(
+                CatalogException.class,
+                () -> glueCatalog.dropPartition(tablePath, partitionSpec, true));
+        createDatabase(tablePath.getDatabaseName());
+        Assertions.assertThrows(
+                CatalogException.class,
+                () -> glueCatalog.dropPartition(tablePath, partitionSpec, true));
+        createTable(tablePath);
+        createPartition(tablePath);
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.dropPartition(tablePath, partitionSpec, true));
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        glueCatalog.dropPartition(
+                                tablePath, new CatalogPartitionSpec(new HashMap<>()), true));
+    }
+
+    @Test
+    public void testAlterPartition() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        createPartition(tablePath);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        CatalogPartition newPartition = new CatalogPartitionImpl(getPartitionSpecParams(), COMMENT);
+
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.alterPartition(tablePath, partitionSpec, newPartition, false));
+
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.alterPartition(tablePath, partitionSpec, newPartition, true));
+
+        Map<String, String> partitionSpecProperties = getPartitionSpecParams();
+        partitionSpecProperties.put("test", "v3");
+
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        glueCatalog.alterPartition(
+                                tablePath,
+                                new CatalogPartitionSpec(partitionSpecProperties),
+                                newPartition,
+                                false));
+
+        ObjectPath tablePath1 = new ObjectPath(DATABASE_1, TABLE_2);
+        createNonPartitionedTable(tablePath1);
+        // since table is not partition , test should throw Catalog Exception
+
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        glueCatalog.alterPartition(
+                                tablePath,
+                                new CatalogPartitionSpec(new HashMap<>()),
+                                new CatalogPartitionImpl(new HashMap<>(), COMMENT),
+                                false));
+
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () ->
+                        glueCatalog.alterPartition(
+                                tablePath,
+                                new CatalogPartitionSpec(null),
+                                new CatalogPartitionImpl(new HashMap<>(), COMMENT),
+                                false));
+        Assertions.assertThrows(
+                NullPointerException.class,
+                () ->
+                        glueCatalog.alterPartition(
+                                tablePath,
+                                new CatalogPartitionSpec(new HashMap<>()),
+                                new CatalogPartitionImpl(null, COMMENT),
+                                false));
+    }
+
+    @Test
+    public void testGetPartition() throws PartitionNotExistException {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        createPartition(tablePath);
+        CatalogPartitionSpec partitionSpec =
+                new CatalogPartitionSpec(GlueCatalogTestUtils.getPartitionSpecParams());
+        CatalogPartition catalogPartition =
+                new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT);
+        Assertions.assertNotNull(catalogPartition);
+
+        Assertions.assertDoesNotThrow(() -> glueCatalog.getPartition(tablePath, partitionSpec));
+        CatalogPartition partition = glueCatalog.getPartition(tablePath, partitionSpec);
+        Assertions.assertNotNull(partition);
+        Assertions.assertNull(partition.getComment());
+        Assertions.assertEquals(
+                GlueCatalogTestUtils.getPartitionSpecParams(), partition.getProperties());
+    }
+
+    @Test
+    public void testPartitionExists() {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        Assertions.assertThrows(
+                CatalogException.class,
+                () -> glueCatalog.partitionExists(tablePath, partitionSpec));
+        createDatabase(tablePath.getDatabaseName());
+        createTable(tablePath);
+        Assertions.assertFalse(glueCatalog.partitionExists(tablePath, partitionSpec));
+        createPartition(tablePath);
+        Assertions.assertTrue(glueCatalog.partitionExists(tablePath, partitionSpec));
+        CatalogPartitionSpec partitionSpecWithNoPartition =
+                new CatalogPartitionSpec(new HashMap<>());
+        Assertions.assertThrows(
+                CatalogException.class,
+                () -> glueCatalog.partitionExists(tablePath, partitionSpecWithNoPartition));
+        Map<String, String> data = new HashMap<>();
+        data.put("col2", "zz1");
+
+        CatalogPartitionSpec partSpecWithPartitionNotExist = new CatalogPartitionSpec(data);
+        Assertions.assertThrows(
+                CatalogException.class,
+                () -> glueCatalog.partitionExists(tablePath, partSpecWithPartitionNotExist));
+    }
+
+    // ---- stats
+
+    @Test
+    public void testAllStatisticsOperationNotSupported()
+            throws PartitionNotExistException, TableNotExistException {
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        CatalogColumnStatistics columnStatistics = new CatalogColumnStatistics(new HashMap<>());
+        CatalogTableStatistics catalogTableStatistics =
+                new CatalogTableStatistics(0L, 0, 0L, 0L, new HashMap<>());
+
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () ->
+                        glueCatalog.alterPartitionColumnStatistics(
+                                tablePath, partitionSpec, columnStatistics, true));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () ->
+                        glueCatalog.alterPartitionColumnStatistics(
+                                tablePath, partitionSpec, columnStatistics, false));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () ->
+                        glueCatalog.alterPartitionStatistics(
+                                tablePath, partitionSpec, catalogTableStatistics, true));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () ->
+                        glueCatalog.alterPartitionStatistics(
+                                tablePath, partitionSpec, catalogTableStatistics, false));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.alterTableColumnStatistics(tablePath, columnStatistics, true));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.alterTableColumnStatistics(tablePath, columnStatistics, false));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.alterTableStatistics(tablePath, catalogTableStatistics, true));
+        Assertions.assertThrows(
+                UnsupportedOperationException.class,
+                () -> glueCatalog.alterTableStatistics(tablePath, catalogTableStatistics, false));
+        Assertions.assertEquals(
+                glueCatalog.getPartitionColumnStatistics(tablePath, partitionSpec),
+                CatalogColumnStatistics.UNKNOWN);
+        Assertions.assertEquals(
+                glueCatalog.getPartitionStatistics(tablePath, partitionSpec),
+                CatalogTableStatistics.UNKNOWN);
+        Assertions.assertEquals(
+                glueCatalog.getTableColumnStatistics(tablePath), CatalogColumnStatistics.UNKNOWN);
+        Assertions.assertEquals(
+                glueCatalog.getTableStatistics(tablePath), CatalogTableStatistics.UNKNOWN);
+    }
+
+    private void createDatabase(String databaseName) {
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createDatabase(databaseName, getDummyCatalogDatabase(), false));
+    }
+
+    private void createTable(ObjectPath tablePath) {
+        CatalogBaseTable baseTable = getDummyCatalogTableWithPartition();
+        Assertions.assertDoesNotThrow(() -> glueCatalog.createTable(tablePath, baseTable, true));
+    }
+
+    private void createNonPartitionedTable(ObjectPath tablePath) {
+        CatalogBaseTable baseTable = getDummyCatalogTable();
+        Assertions.assertDoesNotThrow(() -> glueCatalog.createTable(tablePath, baseTable, true));
+    }
+
+    private void createView(ObjectPath tablePath) {
+        Column column1 = Column.physical(COLUMN_1, DataTypes.STRING());
+        Column column2 = Column.physical(COLUMN_2, DataTypes.STRING());
+        ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2));
+        CatalogView catalogView =
+                CatalogView.of(
+                        Schema.newBuilder()
+                                .column(COLUMN_1, DataTypes.STRING())
+                                .column(COLUMN_2, DataTypes.STRING())
+                                .build(),
+                        COMMENT,
+                        "",
+                        "",
+                        getDummyTableParams());
+
+        ResolvedCatalogView resolvedCatalogView = new ResolvedCatalogView(catalogView, schema);
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createTable(tablePath, resolvedCatalogView, true));
+    }
+
+    private void createFunction(
+            ObjectPath functionPath, FunctionLanguage language, String className) {
+        CatalogFunction catalogFunction =
+                new CatalogFunctionImpl(
+                        className, language, GlueCatalogTestUtils.dummyFlinkResourceUri());
+
+        Assertions.assertDoesNotThrow(
+                () -> glueCatalog.createFunction(functionPath, catalogFunction, true));
+    }
+
+    private void createPartition(ObjectPath tablePath) {
+        CatalogPartitionSpec partitionSpec = new CatalogPartitionSpec(getPartitionSpecParams());
+        CatalogPartition catalogPartition =
+                new CatalogPartitionImpl(GlueCatalogTestUtils.getCatalogPartitionParams(), COMMENT);
+        Assertions.assertDoesNotThrow(
+                () ->
+                        glueCatalog.createPartition(
+                                tablePath, partitionSpec, catalogPartition, false));
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java
new file mode 100644
index 00000000..d2103cf9
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/GlueCatalogTestUtils.java
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogFunctionImpl;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.FunctionLanguage;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+
+import software.amazon.awssdk.http.SdkHttpResponse;
+import software.amazon.awssdk.services.glue.model.CreateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.CreatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.CreateTableRequest;
+import software.amazon.awssdk.services.glue.model.CreateUserDefinedFunctionRequest;
+import software.amazon.awssdk.services.glue.model.Database;
+import software.amazon.awssdk.services.glue.model.Partition;
+import software.amazon.awssdk.services.glue.model.Table;
+import software.amazon.awssdk.services.glue.model.UpdateDatabaseRequest;
+import software.amazon.awssdk.services.glue.model.UpdatePartitionRequest;
+import software.amazon.awssdk.services.glue.model.UpdateTableRequest;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunction;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalogTest.WAREHOUSE_PATH;
+
+/** Contains Utilities for Glue Catalog Tests. */
+public class GlueCatalogTestUtils {
+
+    public static final String DATABASE_DESCRIPTION = "Test database";
+    public static final String DATABASE_1 = "db1";
+    public static final String DATABASE_2 = "db2";
+    public static final String TABLE_1 = "t1";
+    public static final String TABLE_2 = "t2";
+    public static final String TABLE_3 = "t3";
+    public static final String TABLE_4 = "t4";
+    public static final String TABLE_5 = "t5";
+    public static final String VIEW_1 = "v1";
+    public static final String VIEW_2 = "v2";
+    public static final String COLUMN_1 = "name";
+    public static final String COLUMN_2 = "age";
+    public static final String COMMENT = "comment";
+    public static final String EXPANDED_TEXT = "TEST EXPANDED_TEXT";
+    public static final String ORIGINAL_TEXT = "TEST ORIGINAL_TEXT";
+    public static final String FUNCTION_1 = "f1";
+
+    public static Map<String, String> getDatabaseParams() {
+        return new HashMap<String, String>() {
+            {
+                put("key", "value");
+                put("location-uri", WAREHOUSE_PATH);
+            }
+        };
+    }
+
+    public static Map<String, String> getDummyTableParams() {
+        return new HashMap<String, String>() {
+            {
+                put("tableParam1", "v1");
+                put("tableParam2", "v2");
+                put("tableParams3", "v3");
+                put("tableParams4", "v4");
+            }
+        };
+    }
+
+    public static Map<String, String> getPartitionSpecParams() {
+        return new HashMap<String, String>() {
+            {
+                put(COLUMN_1, "v1");
+                put(COLUMN_2, "v2");
+            }
+        };
+    }
+
+    /**
+     * Parameter related to partition.
+     *
+     * @return Partition Properties
+     */
+    public static Map<String, String> getCatalogPartitionParams() {
+        return new HashMap<String, String>() {
+            {
+                put("k1", "v1");
+                put("k2", "v2");
+            }
+        };
+    }
+
+    public static SdkHttpResponse dummySdkHttpResponse(int statusCode) {
+        return SdkHttpResponse.builder().statusCode(statusCode).build();
+    }
+
+    public static Database getDatabaseFromCreateDatabaseRequest(CreateDatabaseRequest request) {
+        return Database.builder()
+                .catalogId(request.catalogId())
+                .name(request.databaseInput().name())
+                .parameters(request.databaseInput().parameters())
+                .description(request.databaseInput().description())
+                .locationUri(request.databaseInput().locationUri())
+                .build();
+    }
+
+    public static Table getTableFromCreateTableRequest(CreateTableRequest request) {
+        return Table.builder()
+                .catalogId(request.catalogId())
+                .databaseName(request.databaseName())
+                .name(request.tableInput().name())
+                .parameters(request.tableInput().parameters())
+                .createdBy(request.tableInput().owner())
+                .description(request.tableInput().description())
+                .createTime(Instant.now())
+                .partitionKeys(request.tableInput().partitionKeys())
+                .storageDescriptor(request.tableInput().storageDescriptor())
+                .tableType(request.tableInput().tableType())
+                .updateTime(Instant.now())
+                .viewExpandedText(request.tableInput().viewExpandedText())
+                .viewOriginalText(request.tableInput().viewOriginalText())
+                .build();
+    }
+
+    public static Table getTableFromUpdateTableRequest(UpdateTableRequest request) {
+        return Table.builder()
+                .catalogId(request.catalogId())
+                .databaseName(request.databaseName())
+                .name(request.tableInput().name())
+                .parameters(request.tableInput().parameters())
+                .createdBy(request.tableInput().owner())
+                .description(request.tableInput().description())
+                .createTime(Instant.now())
+                .partitionKeys(request.tableInput().partitionKeys())
+                .storageDescriptor(request.tableInput().storageDescriptor())
+                .tableType(request.tableInput().tableType())
+                .updateTime(Instant.now())
+                .viewExpandedText(request.tableInput().viewExpandedText())
+                .viewOriginalText(request.tableInput().viewOriginalText())
+                .build();
+    }
+
+    public static String getFullyQualifiedName(String databaseName, String tableName) {
+        return databaseName + "." + tableName;
+    }
+
+    public static Partition getPartitionFromCreatePartitionRequest(CreatePartitionRequest request) {
+        return Partition.builder()
+                .databaseName(request.databaseName())
+                .parameters(request.partitionInput().parameters())
+                .tableName(request.tableName())
+                .storageDescriptor(request.partitionInput().storageDescriptor())
+                .values(request.partitionInput().values())
+                .build();
+    }
+
+    public static Partition getPartitionFromUpdatePartitionRequest(UpdatePartitionRequest request) {
+        return Partition.builder()
+                .storageDescriptor(request.partitionInput().storageDescriptor())
+                .tableName(request.tableName())
+                .databaseName(request.databaseName())
+                .parameters(request.partitionInput().parameters())
+                .values(request.partitionInput().values())
+                .build();
+    }
+
+    public static CatalogDatabase getDummyCatalogDatabase() {
+        return new CatalogDatabaseImpl(getDatabaseParams(), DATABASE_DESCRIPTION);
+    }
+
+    public static UserDefinedFunction getUDFFromCreateUserDefinedFunctionRequest(
+            CreateUserDefinedFunctionRequest request) {
+        return UserDefinedFunction.builder()
+                .functionName(request.functionInput().functionName())
+                .databaseName(request.databaseName())
+                .className(request.functionInput().className())
+                .resourceUris(request.functionInput().resourceUris())
+                .build();
+    }
+
+    public static List<org.apache.flink.table.resource.ResourceUri> dummyFlinkResourceUri() {
+        List<org.apache.flink.table.resource.ResourceUri> resourceUris = new ArrayList<>();
+        resourceUris.add(
+                new org.apache.flink.table.resource.ResourceUri(
+                        org.apache.flink.table.resource.ResourceType.JAR, "URI-JAR"));
+        resourceUris.add(
+                new org.apache.flink.table.resource.ResourceUri(
+                        org.apache.flink.table.resource.ResourceType.FILE, "URI-FILE"));
+        resourceUris.add(
+                new org.apache.flink.table.resource.ResourceUri(
+                        org.apache.flink.table.resource.ResourceType.ARCHIVE, "URI-ARCHIVE"));
+        return resourceUris;
+    }
+
+    public static Database getDatabaseFromUpdateDatabaseRequest(
+            UpdateDatabaseRequest updateDatabaseRequest) {
+        return Database.builder()
+                .catalogId(updateDatabaseRequest.catalogId())
+                .name(updateDatabaseRequest.name())
+                .locationUri(updateDatabaseRequest.databaseInput().locationUri())
+                .description(updateDatabaseRequest.databaseInput().description())
+                .parameters(updateDatabaseRequest.databaseInput().parameters())
+                .build();
+    }
+
+    public static ResolvedCatalogTable getDummyCatalogTable() {
+        Column column1 = Column.physical(COLUMN_1, DataTypes.STRING());
+        Column column2 = Column.physical(COLUMN_2, DataTypes.STRING());
+        ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2));
+        CatalogTable catalogTable =
+                CatalogTable.of(
+                        Schema.newBuilder()
+                                .column(COLUMN_1, DataTypes.STRING())
+                                .column(COLUMN_2, DataTypes.STRING())
+                                .build(),
+                        COMMENT,
+                        new ArrayList<>(),
+                        getDummyTableParams());
+        return new ResolvedCatalogTable(catalogTable, schema);
+    }
+
+    public static CatalogBaseTable getDummyCatalogTableWithPartition() {
+        Column column1 = Column.physical(COLUMN_1, DataTypes.STRING());
+        Column column2 = Column.physical(COLUMN_2, DataTypes.STRING());
+        ResolvedSchema schema = ResolvedSchema.of(Arrays.asList(column1, column2));
+        CatalogTable catalogTable =
+                CatalogTable.of(
+                        Schema.newBuilder()
+                                .column(COLUMN_1, DataTypes.STRING())
+                                .column(COLUMN_2, DataTypes.STRING())
+                                .build(),
+                        COMMENT,
+                        Arrays.asList(COLUMN_1, COLUMN_2),
+                        getDummyTableParams());
+        return new ResolvedCatalogTable(catalogTable, schema);
+    }
+
+    public static CatalogFunction getDummyCatalogFunction() {
+        return new CatalogFunctionImpl("Test Function", FunctionLanguage.JAVA);
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java
new file mode 100644
index 00000000..c159d619
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/TypeMapperTest.java
@@ -0,0 +1,91 @@
+package org.apache.flink.table.catalog.glue;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Unit tests for the {@link TypeMapper} class. */
+public class TypeMapperTest {
+
+    @Test
+    public void testMapFlinkTypeToGlueType_Primitives() {
+        assertEquals("int", TypeMapper.mapFlinkTypeToGlueType(new IntType()));
+        assertEquals("bigint", TypeMapper.mapFlinkTypeToGlueType(new BigIntType()));
+        assertEquals("string", TypeMapper.mapFlinkTypeToGlueType(new VarCharType(255)));
+        assertEquals("boolean", TypeMapper.mapFlinkTypeToGlueType(new BooleanType()));
+        assertEquals("decimal", TypeMapper.mapFlinkTypeToGlueType(new DecimalType(10, 0)));
+        assertEquals("float", TypeMapper.mapFlinkTypeToGlueType(new FloatType()));
+        assertEquals("double", TypeMapper.mapFlinkTypeToGlueType(new DoubleType()));
+        assertEquals("date", TypeMapper.mapFlinkTypeToGlueType(new DateType()));
+        assertEquals("timestamp", TypeMapper.mapFlinkTypeToGlueType(new TimestampType(5)));
+    }
+
+    @Test
+    public void testMapFlinkTypeToGlueType_Array() {
+        LogicalType arrayType = new ArrayType(new VarCharType(255));
+        assertEquals("array<string>", TypeMapper.mapFlinkTypeToGlueType(arrayType));
+    }
+
+    @Test
+    public void testMapFlinkTypeToGlueType_Map() {
+        LogicalType mapType = new MapType(new VarCharType(255), new IntType());
+        assertEquals("map<string,int>", TypeMapper.mapFlinkTypeToGlueType(mapType));
+    }
+
+    @Test
+    public void testMapFlinkTypeToGlueType_Row() {
+        RowType rowType =
+                RowType.of(
+                        new LogicalType[] {new VarCharType(255), new IntType()},
+                        new String[] {"name", "age"});
+        assertEquals("struct<name:string,age:int>", TypeMapper.mapFlinkTypeToGlueType(rowType));
+    }
+
+    @Test
+    public void testGlueTypeToFlinkType_Primitives() {
+        assertEquals(DataTypes.INT(), TypeMapper.glueTypeToFlinkType("int"));
+        assertEquals(DataTypes.BIGINT(), TypeMapper.glueTypeToFlinkType("bigint"));
+        assertEquals(DataTypes.STRING(), TypeMapper.glueTypeToFlinkType("string"));
+        assertEquals(DataTypes.BOOLEAN(), TypeMapper.glueTypeToFlinkType("boolean"));
+        assertEquals(DataTypes.DECIMAL(10, 0), TypeMapper.glueTypeToFlinkType("decimal"));
+        assertEquals(DataTypes.FLOAT(), TypeMapper.glueTypeToFlinkType("float"));
+        assertEquals(DataTypes.DOUBLE(), TypeMapper.glueTypeToFlinkType("double"));
+        assertEquals(DataTypes.DATE(), TypeMapper.glueTypeToFlinkType("date"));
+        assertEquals(DataTypes.TIMESTAMP(5), TypeMapper.glueTypeToFlinkType("timestamp"));
+    }
+
+    @Test
+    public void testGlueTypeToFlinkType_Array() {
+        LogicalType arrayType = new ArrayType(new VarCharType(255));
+        assertEquals("array<string>", TypeMapper.mapFlinkTypeToGlueType(arrayType));
+    }
+
+    @Test
+    public void testGlueTypeToFlinkType_Map() {
+        LogicalType mapType = new MapType(new VarCharType(255), new IntType());
+        assertEquals("map<string,int>", TypeMapper.mapFlinkTypeToGlueType(mapType));
+    }
+
+    @Test
+    public void testGlueTypeToFlinkType_Unsupported() {
+        assertThrows(
+                UnsupportedOperationException.class,
+                () -> TypeMapper.glueTypeToFlinkType("struct<name:string,age:int>"));
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java
new file mode 100644
index 00000000..cc060306
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/constants/AWSGlueConfigConstantsTest.java
@@ -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.
+ */
+
+package org.apache.flink.table.catalog.glue.constants;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.lang.reflect.Field;
+import java.util.HashSet;
+import java.util.Set;
+
+class AWSGlueConfigConstantsTest {
+
+    @Test
+    void testNoDuplicateConstants() {
+        Class<?> glueConfigConstant = AWSGlueConfigConstants.class;
+        Set<String> constants = new HashSet<>();
+        for (Field field : glueConfigConstant.getDeclaredFields()) {
+            String constantValue;
+            try {
+                field.setAccessible(true);
+                constantValue = (String) field.get(null);
+            } catch (IllegalAccessException e) {
+                throw new AssertionError("Error accessing constant field: " + field.getName(), e);
+            }
+            Assertions.assertFalse(
+                    constants.contains(constantValue),
+                    "Duplicate constant found: " + constantValue);
+            constants.add(constantValue);
+        }
+        Assertions.assertEquals(
+                constants.size(),
+                glueConfigConstant.getDeclaredFields().length,
+                "Duplicate Constant Found.");
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java
new file mode 100644
index 00000000..e52e98a0
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/factory/GlueCatalogFactoryTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.factory;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.catalog.glue.GlueCatalogOptions;
+import org.apache.flink.table.factories.CatalogFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_REGION;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.CREDENTIAL_PROVIDER;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.DEFAULT_DATABASE;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_ACCOUNT_ID;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ENDPOINT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.GLUE_CATALOG_ID;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.HTTP_CLIENT_TYPE;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.INPUT_FORMAT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.OUTPUT_FORMAT;
+import static org.apache.flink.table.catalog.glue.GlueCatalogOptions.REGION;
+
+class GlueCatalogFactoryTest extends TestLogger {
+
+    public static GlueCatalogFactory factory;
+
+    @BeforeAll
+    public static void setup() {
+        factory = new GlueCatalogFactory();
+    }
+
+    @Test
+    public void testFactoryIdentifier() {
+        Assertions.assertEquals(GlueCatalogOptions.IDENTIFIER, factory.factoryIdentifier());
+    }
+
+    @Test
+    public void testOptionalOptions() {
+        Set<ConfigOption<?>> configs = factory.optionalOptions();
+        Assertions.assertNotNull(configs);
+        Assertions.assertEquals(9, configs.size());
+        Assertions.assertTrue(configs.contains(INPUT_FORMAT));
+        Assertions.assertTrue(configs.contains(OUTPUT_FORMAT));
+        Assertions.assertTrue(configs.contains(GLUE_CATALOG_ENDPOINT));
+        Assertions.assertTrue(configs.contains(GLUE_ACCOUNT_ID));
+        Assertions.assertTrue(configs.contains(GLUE_CATALOG_ID));
+        Assertions.assertTrue(configs.contains(DEFAULT_DATABASE));
+        Assertions.assertTrue(configs.contains(HTTP_CLIENT_TYPE));
+        Assertions.assertTrue(configs.contains(REGION));
+        Assertions.assertTrue(configs.contains(CREDENTIAL_PROVIDER));
+    }
+
+    @Test
+    public void testGetRequiredOptions() {
+        Set<ConfigOption<?>> configs = factory.requiredOptions();
+        Assertions.assertNotNull(configs);
+        Assertions.assertEquals(new HashSet<>(), configs);
+    }
+
+    @Test
+    public void testCreateCatalog() {
+        Map<String, String> options = new HashMap<>();
+        ReadableConfig configs = new Configuration();
+        CatalogFactory.Context context =
+                new FactoryUtil.DefaultCatalogContext(
+                        "TestContext", options, configs, ClassLoader.getSystemClassLoader());
+        Assertions.assertThrows(NullPointerException.class, () -> factory.createCatalog(context));
+        options.put(AWS_REGION, "us-east-1");
+        Assertions.assertDoesNotThrow(() -> factory.createCatalog(context));
+    }
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java
new file mode 100644
index 00000000..7daeb9de
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueCatalogOptionsUtilsTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.util;
+
+import org.junit.jupiter.api.Test;
+
+class GlueCatalogOptionsUtilsTest {
+
+    @Test
+    void testGetValidatedConfigurations() {}
+
+    @Test
+    void testGetProcessedResolvedOptions() {}
+
+    @Test
+    void testGetNonValidatedPrefixes() {}
+}
diff --git a/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java
new file mode 100644
index 00000000..d667aad2
--- /dev/null
+++ b/flink-catalog-aws/flink-catalog-aws-glue/src/test/java/org/apache/flink/table/catalog/glue/util/GlueUtilsTest.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.catalog.glue.util;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogFunctionImpl;
+import org.apache.flink.table.catalog.FunctionLanguage;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.glue.constants.GlueCatalogConstants;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.utils.ResolvedExpressionMock;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import software.amazon.awssdk.services.glue.model.Database;
+import software.amazon.awssdk.services.glue.model.UserDefinedFunction;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.DATABASE_1;
+import static org.apache.flink.table.catalog.glue.GlueCatalogTestUtils.TABLE_1;
+
+/** Test methods in GlueUtils . */
+public class GlueUtilsTest {
+
+    private static final String WAREHOUSE_PATH = "s3://bucket";
+
+    @Test
+    public void testGetGlueConventionalName() {
+        String name = "MyName";
+        Assertions.assertEquals("myname", GlueUtils.getGlueConventionalName(name));
+        String name1 = "Mtx@ndfv";
+        Assertions.assertThrows(IllegalArgumentException.class, () -> GlueUtils.validate(name1));
+    }
+
+    @Test
+    public void testExtractDatabaseLocation() {
+        HashMap<String, String> propertiesWithLocationUri =
+                new HashMap<String, String>() {
+                    {
+                        put(GlueCatalogConstants.LOCATION_URI, "s3://some-path/myDb/");
+                        put("k1", "v1");
+                    }
+                };
+
+        String location =
+                GlueUtils.extractDatabaseLocation(
+                        propertiesWithLocationUri, DATABASE_1, WAREHOUSE_PATH);
+        Assertions.assertEquals("s3://some-path/myDb/", location);
+
+        String newLocation =
+                GlueUtils.extractDatabaseLocation(
+                        propertiesWithLocationUri, DATABASE_1, WAREHOUSE_PATH);
+        Assertions.assertNotEquals("s3://some-path/myDb/", newLocation);
+        Assertions.assertEquals(
+                WAREHOUSE_PATH + GlueCatalogConstants.LOCATION_SEPARATOR + DATABASE_1, newLocation);
+        newLocation =
+                GlueUtils.extractDatabaseLocation(new HashMap<>(), DATABASE_1, WAREHOUSE_PATH);
+        Assertions.assertEquals(
+                newLocation, WAREHOUSE_PATH + GlueCatalogConstants.LOCATION_SEPARATOR + DATABASE_1);
+    }
+
+    @Test
+    public void testExtractTableLocation() {
+        Map<String, String> propertiesWithLocationUri =
+                new HashMap<String, String>() {
+                    {
+                        put(GlueCatalogConstants.LOCATION_URI, "s3://some-path/myDb/myTable/");
+                        put("k1", "v1");
+                    }
+                };
+        ObjectPath tablePath = new ObjectPath(DATABASE_1, TABLE_1);
+        String location =
+                GlueUtils.extractTableLocation(
+                        propertiesWithLocationUri, tablePath, WAREHOUSE_PATH);
+        Assertions.assertEquals("s3://some-path/myDb/myTable/", location);
+
+        String newLocation =
+                GlueUtils.extractTableLocation(
+                        propertiesWithLocationUri, tablePath, WAREHOUSE_PATH);
+        Assertions.assertNotEquals("s3://some-path/myDb/myTable", newLocation);
+        Assertions.assertEquals(
+                WAREHOUSE_PATH
+                        + GlueCatalogConstants.LOCATION_SEPARATOR
+                        + DATABASE_1
+                        + GlueCatalogConstants.LOCATION_SEPARATOR
+                        + TABLE_1,
+                newLocation);
+    }
+
+    @Test
+    public void testGetCatalogDatabase() {
+        Map<String, String> params =
+                new HashMap<String, String>() {
+                    {
+                        put("k1", "v1");
+                        put("k2", "v2");
+                    }
+                };
+        String description = "Test description";
+        Database database = Database.builder().parameters(params).description(description).build();
+        CatalogDatabase catalogDatabase = GlueUtils.getCatalogDatabase(database);
+        Assertions.assertInstanceOf(CatalogDatabase.class, catalogDatabase);
+        Assertions.assertEquals(catalogDatabase.getProperties(), params);
+        Assertions.assertEquals(catalogDatabase.getDescription().orElse(null), description);
+    }
+
+    @Test
+    public void testGetCatalogFunctionClassName() {
+        UserDefinedFunction.Builder udfBuilder =
+                UserDefinedFunction.builder().functionName("Dummy").databaseName(DATABASE_1);
+        UserDefinedFunction udf1 = udfBuilder.className("org.test.Class").build();
+        Assertions.assertThrows(
+                org.apache.flink.table.api.ValidationException.class,
+                () -> GlueUtils.getCatalogFunctionClassName(udf1));
+        String className = GlueUtils.getGlueFunctionClassName(new CatalogFunctionImpl("TestClass"));
+        UserDefinedFunction udf2 = udfBuilder.className(className).build();
+        Assertions.assertDoesNotThrow(() -> GlueUtils.getCatalogFunctionClassName(udf2));
+    }
+
+    @Test
+    public void testGetFunctionalLanguage() {
+        UserDefinedFunction.Builder udfBuilder =
+                UserDefinedFunction.builder().functionName("Dummy").databaseName(DATABASE_1);
+        Assertions.assertThrows(
+                CatalogException.class,
+                () ->
+                        GlueUtils.getFunctionalLanguage(
+                                udfBuilder.className("org.test.Class").build()));
+        String className = GlueUtils.getGlueFunctionClassName(new CatalogFunctionImpl("TestClass"));
+        UserDefinedFunction udf1 = udfBuilder.className(className).build();
+        FunctionLanguage functionLanguage = GlueUtils.getFunctionalLanguage(udf1);
+        Assertions.assertEquals(functionLanguage, FunctionLanguage.JAVA);
+    }
+
+    @Test
+    public void testExtractTableOwner() {
+        Map<String, String> properties =
+                new HashMap<String, String>() {
+                    {
+                        put("k1", "v1");
+                        put("k2", "v2");
+                    }
+                };
+
+        Assertions.assertNull(GlueUtils.extractTableOwner(properties));
+        properties.put(GlueCatalogConstants.TABLE_OWNER, "testOwner");
+        Assertions.assertEquals(GlueUtils.extractTableOwner(properties), "testOwner");
+    }
+
+    @Test
+    public void testExpressionString() {
+        Expression expression = ResolvedExpressionMock.of(DataTypes.INT(), "column1");
+        Assertions.assertEquals("column1", GlueUtils.getExpressionString(expression));
+    }
+}
diff --git a/flink-catalog-aws/pom.xml b/flink-catalog-aws/pom.xml
new file mode 100644
index 00000000..ce2f0b7f
--- /dev/null
+++ b/flink-catalog-aws/pom.xml
@@ -0,0 +1,38 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <artifactId>flink-connector-aws</artifactId>
+        <groupId>org.apache.flink</groupId>
+        <version>4.4-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>flink-catalog-aws-parent</artifactId>
+    <name>Flink : Catalog : AWS : Parent</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>flink-catalog-aws-glue</module>
+    </modules>
+
+</project>
\ No newline at end of file
diff --git a/flink-connector-aws-base/pom.xml b/flink-connector-aws-base/pom.xml
index f06cc690..41a0ce2c 100644
--- a/flink-connector-aws-base/pom.xml
+++ b/flink-connector-aws-base/pom.xml
@@ -94,6 +94,10 @@ under the License.
             <artifactId>flink-architecture-tests-test</artifactId>
             <scope>test</scope>
         </dependency>
+        <dependency>
+            <groupId>software.amazon.awssdk</groupId>
+            <artifactId>url-connection-client</artifactId>
+        </dependency>
 
     </dependencies>
 
diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java
index aea2bfe7..392ff6b4 100644
--- a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java
+++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/config/AWSConfigConstants.java
@@ -154,6 +154,54 @@ public enum CredentialProvider {
     /** Read Request timeout for {@link SdkAsyncHttpClient}. */
     public static final String HTTP_CLIENT_READ_TIMEOUT_MILLIS = "aws.http-client.read-timeout";
 
+    /**
+     * The type of {@link software.amazon.awssdk.http.SdkHttpClient}. If set, all AWS clients will
+     * use this specified HTTP client. If not set, HTTP_CLIENT_TYPE_DEFAULT will be used. For
+     * specific types supported, see HTTP_CLIENT_TYPE_* defined below.
+     */
+    public static final String HTTP_CLIENT_TYPE = "http-client.type";
+
+    // ---- glue configs
+
+    /**
+     * Used to configure the connection timeout in milliseconds for {@link
+     * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when
+     * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE
+     *
+     * <p>For more details, see <a
+     * href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html">...</a>
+     */
+    public static final String HTTP_CLIENT_CONNECTION_TIMEOUT_MS =
+            "http-client.connection-timeout-ms";
+
+    /**
+     * Used to configure the max connections number for {@link
+     * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when
+     * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE
+     *
+     * <p>For more details, see <a
+     * href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html">...</a>
+     */
+    public static final String HTTP_CLIENT_APACHE_MAX_CONNECTIONS =
+            "http-client.apache.max-connections";
+
+    /**
+     * Used to configure the socket timeout in milliseconds for {@link
+     * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when
+     * {@link #HTTP_CLIENT_TYPE} is set to HTTP_CLIENT_TYPE_APACHE
+     *
+     * <p>For more details, see <a
+     * href="https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html">...</a>
+     */
+    public static final String HTTP_CLIENT_SOCKET_TIMEOUT_MS = "http-client.socket-timeout-ms";
+
+    public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection";
+
+    /**
+     * {@link software.amazon.awssdk.http.apache.ApacheHttpClient} will be used as the HTTP Client.
+     */
+    public static final String CLIENT_TYPE_APACHE = "apache";
+
     public static String accessKeyId(String prefix) {
         return prefix + ".basic.accesskeyid";
     }
diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java
new file mode 100644
index 00000000..d3a6bc0f
--- /dev/null
+++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtils.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.flink.connector.aws.table.util;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import org.apache.flink.connector.base.table.options.ConfigurationValidator;
+import org.apache.flink.connector.base.table.options.TableOptionsUtils;
+import org.apache.flink.connector.base.table.util.ConfigurationValidatorUtil;
+
+import software.amazon.awssdk.http.Protocol;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/** Class for handling AWS HTTP Client config options. */
+@PublicEvolving
+public class HttpClientOptionUtils implements TableOptionsUtils, ConfigurationValidator {
+    public static final String CLIENT_PREFIX = "http-client.";
+    private static final String CLIENT_HTTP_PROTOCOL_VERSION_OPTION = "protocol.version";
+    private static final String CLIENT_HTTP_MAX_CONNECTION_TIMEOUT_MS = "connection-timeout-ms";
+    private static final String CLIENT_HTTP_MAX_SOCKET_TIMEOUT_MS = "socket-timeout-ms";
+    private static final String APACHE_MAX_CONNECTIONS = "apache.max-connections";
+
+    private final List<String> allowedClientTypes;
+    private final Map<String, String> resolvedOptions;
+
+    public HttpClientOptionUtils(String[] allowedClientTypes, Map<String, String> resolvedOptions) {
+        this.allowedClientTypes = Arrays.asList(allowedClientTypes);
+        this.resolvedOptions = resolvedOptions;
+    }
+
+    @Override
+    public Properties getValidatedConfigurations() {
+        Properties clientConfigurations = new Properties();
+        clientConfigurations.putAll(getProcessedResolvedOptions());
+        validateClientType(clientConfigurations);
+        validateConfigurations(clientConfigurations);
+        return clientConfigurations;
+    }
+
+    @Override
+    public Map<String, String> getProcessedResolvedOptions() {
+        Map<String, String> mappedResolvedOptions = new HashMap<>();
+        for (String key : resolvedOptions.keySet()) {
+            if (key.startsWith(CLIENT_PREFIX)) {
+                mappedResolvedOptions.put(translateClientKeys(key), resolvedOptions.get(key));
+            }
+        }
+        return mappedResolvedOptions;
+    }
+
+    @Override
+    public List<String> getNonValidatedPrefixes() {
+        return Collections.singletonList(CLIENT_PREFIX);
+    }
+
+    private static String translateClientKeys(String key) {
+        String truncatedKey = key.substring(CLIENT_PREFIX.length());
+        switch (truncatedKey) {
+            case CLIENT_HTTP_PROTOCOL_VERSION_OPTION:
+                return AWSConfigConstants.HTTP_PROTOCOL_VERSION;
+            case CLIENT_HTTP_MAX_CONNECTION_TIMEOUT_MS:
+                return AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS;
+            case CLIENT_HTTP_MAX_SOCKET_TIMEOUT_MS:
+                return AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS;
+            case APACHE_MAX_CONNECTIONS:
+                return AWSConfigConstants.HTTP_CLIENT_APACHE_MAX_CONNECTIONS;
+            default:
+                return "aws.http-client." + truncatedKey;
+        }
+    }
+
+    private void validateConfigurations(Properties config) {
+        ConfigurationValidatorUtil.validateOptionalPositiveIntProperty(
+                config,
+                AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS,
+                "Invalid value given for HTTP connection timeout. Must be positive integer.");
+        ConfigurationValidatorUtil.validateOptionalPositiveIntProperty(
+                config,
+                AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS,
+                "Invalid value given for HTTP socket read timeout. Must be positive integer.");
+        ConfigurationValidatorUtil.validateOptionalPositiveIntProperty(
+                config,
+                AWSConfigConstants.HTTP_CLIENT_APACHE_MAX_CONNECTIONS,
+                "Invalid value for max number of Connection. Must be positive integer.");
+        ConfigurationValidatorUtil.validateOptionalPositiveIntProperty(
+                config,
+                AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY,
+                "Invalid value given for HTTP client max concurrency. Must be positive integer.");
+        validateOptionalHttpProtocolProperty(config);
+    }
+
+    private void validateClientType(Properties config) {
+        if (config.containsKey(AWSConfigConstants.HTTP_CLIENT_TYPE)
+                && !allowedClientTypes.contains(
+                        config.getProperty(AWSConfigConstants.HTTP_CLIENT_TYPE))) {
+            throw new IllegalArgumentException("Invalid Http Client Type.");
+        }
+    }
+
+    private void validateOptionalHttpProtocolProperty(Properties config) {
+        if (config.containsKey(AWSConfigConstants.HTTP_PROTOCOL_VERSION)) {
+            try {
+                Protocol.valueOf(config.getProperty(AWSConfigConstants.HTTP_PROTOCOL_VERSION));
+            } catch (IllegalArgumentException e) {
+                throw new IllegalArgumentException(
+                        "Invalid value given for HTTP protocol. Must be HTTP1_1 or HTTP2.");
+            }
+        }
+    }
+}
diff --git a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java
index cea82483..4a7d0402 100644
--- a/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java
+++ b/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java
@@ -40,6 +40,7 @@
 import software.amazon.awssdk.http.async.SdkAsyncHttpClient;
 import software.amazon.awssdk.http.nio.netty.Http2Configuration;
 import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient;
+import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient;
 import software.amazon.awssdk.profiles.ProfileFile;
 import software.amazon.awssdk.regions.Region;
 import software.amazon.awssdk.services.sts.StsClient;
@@ -267,7 +268,7 @@ private static AwsCredentialsProvider getAssumeRoleCredentialProvider(
                                         configProps.getProperty(
                                                 AWSConfigConstants.externalId(configPrefix)))
                                 .build())
-                .stsClient(stsClientBuilder.build())
+                .stsClient(stsClientBuilder.httpClientBuilder(ApacheHttpClient.builder()).build())
                 .build();
     }
 
@@ -355,12 +356,52 @@ public static SdkAsyncHttpClient createAsyncHttpClient(
         return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS));
     }
 
+    public static SdkHttpClient createSyncHttpClient(
+            final Properties configProperties, final ApacheHttpClient.Builder httpClientBuilder) {
+        final AttributeMap.Builder clientConfiguration = AttributeMap.builder();
+
+        clientConfiguration.put(SdkHttpConfigurationOption.TCP_KEEPALIVE, true);
+        Optional.ofNullable(
+                        configProperties.getProperty(
+                                SdkHttpConfigurationOption.TCP_KEEPALIVE.name()))
+                .map(Boolean::parseBoolean)
+                .ifPresent(
+                        booleanValue ->
+                                clientConfiguration.put(
+                                        SdkHttpConfigurationOption.TCP_KEEPALIVE, booleanValue));
+        Optional.ofNullable(
+                        configProperties.getProperty(
+                                AWSConfigConstants.HTTP_CLIENT_CONNECTION_TIMEOUT_MS))
+                .map(Long::parseLong)
+                .ifPresent(
+                        longValue ->
+                                clientConfiguration.put(
+                                        SdkHttpConfigurationOption.CONNECTION_TIMEOUT,
+                                        Duration.ofMillis(longValue)));
+        Optional.ofNullable(
+                        configProperties.getProperty(
+                                AWSConfigConstants.HTTP_CLIENT_SOCKET_TIMEOUT_MS))
+                .map(Long::parseLong)
+                .ifPresent(
+                        longValue ->
+                                clientConfiguration.put(
+                                        SdkHttpConfigurationOption.READ_TIMEOUT,
+                                        Duration.ofMillis(longValue)));
+
+        return createSyncHttpClient(clientConfiguration.build(), httpClientBuilder);
+    }
+
     public static SdkHttpClient createSyncHttpClient(
             final AttributeMap config, final ApacheHttpClient.Builder httpClientBuilder) {
         httpClientBuilder.connectionAcquisitionTimeout(CONNECTION_ACQUISITION_TIMEOUT);
         return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS));
     }
 
+    public static SdkHttpClient createSyncHttpClient(
+            final AttributeMap config, final UrlConnectionHttpClient.Builder httpClientBuilder) {
+        return httpClientBuilder.buildWithDefaults(config.merge(HTTP_CLIENT_DEFAULTS));
+    }
+
     /**
      * Extract region from resource ARN.
      *
diff --git a/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java b/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java
new file mode 100644
index 00000000..29ba3897
--- /dev/null
+++ b/flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/table/util/HttpClientOptionUtilsTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.aws.table.util;
+
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+class HttpClientOptionUtilsTest {
+
+    private static final String[] ALLOWED_GLUE_HTTP_CLIENTS =
+            new String[] {
+                AWSConfigConstants.CLIENT_TYPE_URLCONNECTION, AWSConfigConstants.CLIENT_TYPE_APACHE
+            };
+
+    @Test
+    public void testGoodHttpClientOptionsMapping() {
+        HttpClientOptionUtils httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, getDefaultClientOptions());
+
+        Map<String, String> expectedConfigurations = getDefaultExpectedClientOptions();
+        Map<String, String> actualConfigurations =
+                httpClientOptionUtils.getProcessedResolvedOptions();
+
+        Assertions.assertEquals(expectedConfigurations, actualConfigurations);
+    }
+
+    @Test
+    void testHttpClientOptionsUtilsFilteringNonPrefixedOptions() {
+        Map<String, String> defaultClientOptions = getDefaultClientOptions();
+        defaultClientOptions.put("aws.not.http-client.dummy.option", "someValue");
+
+        HttpClientOptionUtils httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultClientOptions);
+
+        Map<String, String> expectedConfigurations = getDefaultExpectedClientOptions();
+        Map<String, String> actualConfigurations =
+                httpClientOptionUtils.getProcessedResolvedOptions();
+
+        Assertions.assertEquals(expectedConfigurations, actualConfigurations);
+    }
+
+    @Test
+    void testHttpClientOptionsUtilsExtractingCorrectConfiguration() {
+        HttpClientOptionUtils httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, getDefaultClientOptions());
+
+        Properties expectedConfigurations = getDefaultExpectedClientConfigs();
+        Properties actualConfigurations = httpClientOptionUtils.getValidatedConfigurations();
+
+        Assertions.assertEquals(expectedConfigurations, actualConfigurations);
+    }
+
+    @Test
+    void testHttpClientOptionsUtilsFailOnInvalidMaxConcurrency() {
+        Map<String, String> defaultClientOptions = getDefaultClientOptions();
+        defaultClientOptions.put("http-client.max-concurrency", "invalid-integer");
+
+        HttpClientOptionUtils httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultClientOptions);
+
+        Assertions.assertThrows(
+                IllegalArgumentException.class, httpClientOptionUtils::getValidatedConfigurations);
+    }
+
+    @Test
+    void testHttpClientOptionsUtilsFailOnInvalidHttpProtocol() {
+        Map<String, String> defaultProperties = getDefaultClientOptions();
+        defaultProperties.put("http-client.protocol.version", "invalid-http-protocol");
+
+        HttpClientOptionUtils httpClientOptionUtils =
+                new HttpClientOptionUtils(ALLOWED_GLUE_HTTP_CLIENTS, defaultProperties);
+
+        Assertions.assertThrows(
+                IllegalArgumentException.class, httpClientOptionUtils::getValidatedConfigurations);
+    }
+
+    private static Map<String, String> getDefaultClientOptions() {
+        Map<String, String> defaultGlueClientOptions = new HashMap<String, String>();
+        defaultGlueClientOptions.put("region", "us-east-1");
+        defaultGlueClientOptions.put("http-client.max-concurrency", "10000");
+        defaultGlueClientOptions.put("http-client.protocol.version", "HTTP2");
+        return defaultGlueClientOptions;
+    }
+
+    private static Map<String, String> getDefaultExpectedClientOptions() {
+        Map<String, String> defaultExpectedGlueClientConfigurations = new HashMap<String, String>();
+        defaultExpectedGlueClientConfigurations.put(
+                AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY, "10000");
+        defaultExpectedGlueClientConfigurations.put(
+                AWSConfigConstants.HTTP_PROTOCOL_VERSION, "HTTP2");
+        return defaultExpectedGlueClientConfigurations;
+    }
+
+    private static Properties getDefaultExpectedClientConfigs() {
+        Properties defaultExpectedGlueClientConfigurations = new Properties();
+        defaultExpectedGlueClientConfigurations.put(
+                AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY, "10000");
+        defaultExpectedGlueClientConfigurations.put(
+                AWSConfigConstants.HTTP_PROTOCOL_VERSION, "HTTP2");
+        return defaultExpectedGlueClientConfigurations;
+    }
+}
diff --git a/pom.xml b/pom.xml
index 4c69059e..f1d20347 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,10 +80,13 @@ under the License.
 
     <modules>
         <module>flink-connector-aws-base</module>
+
         <module>flink-connector-aws</module>
         <module>flink-formats-aws</module>
         <module>flink-python</module>
+        <module>flink-catalog-aws</module>
         <module>flink-connector-aws-e2e-tests</module>
+
     </modules>
 
     <dependencies>
@@ -495,4 +498,4 @@ under the License.
             </plugin>
         </plugins>
     </build>
-</project>
+</project>
\ No newline at end of file