From 9aaaeb34741e41471e2a2404fff31567c5cb6a61 Mon Sep 17 00:00:00 2001 From: fanng Date: Fri, 28 Jun 2024 12:46:09 +0800 Subject: [PATCH 01/11] support split iceberg rest service --- .../workflows/backend-integration-test.yml | 2 + LICENSE | 4 +- bin/gravitino.sh | 2 + bin/iceberg-rest-server.sh | 206 ++++++++++++++++++ build.gradle.kts | 72 +++++- .../build.gradle.kts | 44 +--- .../iceberg/IcebergCatalogOperations.java | 8 +- .../IcebergCatalogPropertiesMetadata.java | 6 +- .../iceberg/ops/IcebergTableOpsHelper.java | 26 +-- .../org.apache.gravitino.CatalogProvider | 2 +- .../lakehouse/iceberg/TestIcebergCatalog.java | 2 +- .../test/CatalogIcebergBaseIT.java | 4 +- .../test/CatalogIcebergKerberosHiveIT.java | 12 +- .../test/CatalogIcebergRestIT.java | 10 +- .../test/TestMultipleJDBCLoad.java | 2 +- .../iceberg/ops/TestIcebergTableUpdate.java | 5 +- conf/gravitino.conf.template | 11 +- conf/iceberg-rest-server.conf.template | 47 ++++ conf/log4j2.properties.template | 5 +- .../org/apache/gravitino/GravitinoEnv.java | 19 +- .../auxiliary/AuxiliaryServiceManager.java | 37 +++- .../TestAuxiliaryServiceManager.java | 67 +++++- iceberg/iceberg-common/build.gradle.kts | 87 ++++++++ .../iceberg/common}/ClosableHiveCatalog.java | 2 +- .../common}/IcebergCatalogBackend.java | 2 +- .../iceberg/common}/IcebergConfig.java | 26 +-- .../iceberg/common/IcebergConstants.java | 41 ++++ .../authentication/AuthenticationConfig.java | 2 +- .../kerberos/FetchFileUtils.java | 2 +- .../kerberos/HiveBackendProxy.java | 7 +- .../kerberos/KerberosClient.java | 3 +- .../kerberos/KerberosConfig.java | 5 +- .../iceberg/common}/ops/IcebergTableOps.java | 34 ++- .../common}/utils/IcebergCatalogUtil.java | 35 +-- .../utils}/IcebergHiveCachedClientPool.java | 6 +- .../iceberg/common}/TestIcebergConfig.java | 3 +- .../common}/utils/TestIcebergCatalogUtil.java | 14 +- iceberg/iceberg-rest-server/build.gradle.kts | 171 +++++++++++++++ .../apache/gravitino/iceberg/RESTService.java | 27 ++- .../service}/IcebergExceptionMapper.java | 4 +- .../iceberg/service}/IcebergObjectMapper.java | 4 +- .../service}/IcebergObjectMapperProvider.java | 3 +- .../iceberg/service}/IcebergRestUtils.java | 3 +- .../service}/metrics/DummyMetricsStore.java | 4 +- .../metrics/IcebergMetricsFormatter.java | 6 +- .../metrics/IcebergMetricsManager.java | 17 +- .../service}/metrics/IcebergMetricsStore.java | 4 +- .../rest/IcebergConfigOperations.java | 6 +- .../rest/IcebergNamespaceOperations.java | 7 +- .../service}/rest/IcebergTableOperations.java | 12 +- .../rest/IcebergTableRenameOperations.java | 7 +- .../gravitino/server/IcebergRESTServer.java | 105 +++++++++ ...vitino.auxiliary.GravitinoAuxiliaryService | 19 ++ .../src/main/resources/core-site.xml.template | 21 ++ .../src/main/resources/hdfs-site.xml.template | 20 ++ .../test/IcebergRESTHiveCatalogIT.java | 36 +-- .../test/IcebergRESTJdbcCatalogIT.java | 48 +--- .../test/IcebergRESTMemoryCatalogIT.java | 42 ++++ .../test/IcebergRESTServiceBaseIT.java | 54 ++--- .../test/IcebergRESTServiceIT.java | 37 +--- .../test/util/IcebergRESTServerManager.java | 166 ++++++++++++++ .../IcebergRESTServerManagerForDeploy.java | 68 ++++++ .../IcebergRESTServerManagerForEmbedded.java | 77 +++++++ .../service}/TestIcebergExceptionMapper.java | 3 +- .../service}/metrics/MemoryMetricsStore.java | 5 +- .../metrics/TestIcebergMetricsManager.java | 14 +- .../service}/rest/IcebergRestTestUtil.java | 14 +- .../service}/rest/IcebergTestBase.java | 17 +- .../service}/rest/TestIcebergConfig.java | 4 +- .../rest/TestIcebergNamespaceOperations.java | 3 +- .../rest/TestIcebergTableOperations.java | 4 +- .../src/test/resources/log4j2.properties | 73 +++++++ .../integration/test/MiniGravitino.java | 16 +- .../integration/test/util/AbstractIT.java | 2 +- server-common/build.gradle.kts | 7 + .../apache/gravitino/server/ServerConfig.java | 18 ++ .../server/web/JettyServerConfig.java | 1 + .../gravitino/server/TestServerConfig.java | 6 +- .../gravitino/server/GravitinoServer.java | 2 +- settings.gradle.kts | 2 + .../integration/test/SparkEnvIT.java | 11 +- spark-connector/v3.3/spark/build.gradle.kts | 1 + spark-connector/v3.4/spark/build.gradle.kts | 1 + spark-connector/v3.5/spark/build.gradle.kts | 1 + 84 files changed, 1601 insertions(+), 434 deletions(-) create mode 100755 bin/iceberg-rest-server.sh create mode 100644 conf/iceberg-rest-server.conf.template create mode 100644 iceberg/iceberg-common/build.gradle.kts rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/catalog => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/ClosableHiveCatalog.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/IcebergCatalogBackend.java (94%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/IcebergConfig.java (84%) create mode 100644 iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/authentication/AuthenticationConfig.java (98%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/authentication/kerberos/FetchFileUtils.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/authentication/kerberos/HiveBackendProxy.java (94%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/authentication/kerberos/KerberosClient.java (98%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/authentication/kerberos/KerberosConfig.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/ops/IcebergTableOps.java (91%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common}/utils/IcebergCatalogUtil.java (84%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils}/IcebergHiveCachedClientPool.java (97%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common}/TestIcebergConfig.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg => iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common}/utils/TestIcebergCatalogUtil.java (81%) create mode 100644 iceberg/iceberg-rest-server/build.gradle.kts rename catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergRESTService.java => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java (84%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/IcebergExceptionMapper.java (98%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/IcebergObjectMapper.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/IcebergObjectMapperProvider.java (94%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/IcebergRestUtils.java (97%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/metrics/DummyMetricsStore.java (94%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/metrics/IcebergMetricsFormatter.java (90%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/metrics/IcebergMetricsManager.java (91%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/metrics/IcebergMetricsStore.java (96%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/rest/IcebergConfigOperations.java (92%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/rest/IcebergNamespaceOperations.java (95%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/rest/IcebergTableOperations.java (95%) rename {catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service}/rest/IcebergTableRenameOperations.java (90%) create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java create mode 100644 iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService create mode 100644 iceberg/iceberg-rest-server/src/main/resources/core-site.xml.template create mode 100644 iceberg/iceberg-rest-server/src/main/resources/hdfs-site.xml.template rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino}/iceberg/integration/test/IcebergRESTHiveCatalogIT.java (63%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino}/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java (57%) create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTMemoryCatalogIT.java rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino}/iceberg/integration/test/IcebergRESTServiceBaseIT.java (79%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino}/iceberg/integration/test/IcebergRESTServiceIT.java (94%) create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java create mode 100644 iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/TestIcebergExceptionMapper.java (97%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/metrics/MemoryMetricsStore.java (93%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/metrics/TestIcebergMetricsManager.java (89%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/rest/IcebergRestTestUtil.java (86%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/rest/IcebergTestBase.java (90%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/rest/TestIcebergConfig.java (97%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/rest/TestIcebergNamespaceOperations.java (99%) rename {catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web => iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service}/rest/TestIcebergTableOperations.java (99%) create mode 100644 iceberg/iceberg-rest-server/src/test/resources/log4j2.properties rename {server => server-common}/src/main/java/org/apache/gravitino/server/ServerConfig.java (70%) rename {server => server-common}/src/test/java/org/apache/gravitino/server/TestServerConfig.java (95%) diff --git a/.github/workflows/backend-integration-test.yml b/.github/workflows/backend-integration-test.yml index 437acbd02fa..0bf06a55d75 100644 --- a/.github/workflows/backend-integration-test.yml +++ b/.github/workflows/backend-integration-test.yml @@ -37,6 +37,7 @@ jobs: - gradle/** - integration-test/** - integration-test-common/** + - iceberg/** - meta/** - server/** - server-common/** @@ -104,6 +105,7 @@ jobs: name: integrate-test-reports-${{ matrix.java-version }}-${{ matrix.test-mode }}-${{ matrix.backend }} path: | build/reports + iceberg/iceberg-rest-server/build/*.log integration-test/build/*.log integration-test/build/*.tar integration-test/build/trino-ci-container-log/hive/*.* diff --git a/LICENSE b/LICENSE index de48fa68ed5..169b5ef9964 100644 --- a/LICENSE +++ b/LICENSE @@ -232,10 +232,10 @@ ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/miniHMS/MiniHiveMetastoreService.java ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/miniHMS/ScriptRunner.java ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/TestCachedClientPool.java - ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergExceptionMapper.java + ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/org.apache.service.iceberg.gravitino.IcebergExceptionMapper.java ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/converter/DescribeIcebergSortOrderVisitor.java ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/IcebergTablePropertiesUtil.java - ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergHiveCachedClientPool.java + ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool.java ./clients/client-java/src/main/java/org/apache/gravitino/client/HTTPClient.java ./clients/client-java/src/main/java/org/apache/gravitino/client/RESTClient.java ./clients/client-java/src/test/java/org/apache/gravitino/client/TestHTTPClient.java diff --git a/bin/gravitino.sh b/bin/gravitino.sh index 93fd7e4b8be..0ba5364994e 100755 --- a/bin/gravitino.sh +++ b/bin/gravitino.sh @@ -144,10 +144,12 @@ function stop() { HOSTNAME=$(hostname) GRAVITINO_OUTFILE="${GRAVITINO_LOG_DIR}/gravitino-server.out" GRAVITINO_SERVER_NAME=org.apache.gravitino.server.GravitinoServer +GRAVITINO_SIMPLE_SERVER_NAME=gravitino-server JAVA_OPTS+=" -Dfile.encoding=UTF-8" JAVA_OPTS+=" -Dlog4j2.configurationFile=file://${GRAVITINO_CONF_DIR}/log4j2.properties" JAVA_OPTS+=" -Dgravitino.log.path=${GRAVITINO_LOG_DIR} ${GRAVITINO_MEM}" +JAVA_OPTS+=" -Dgravitino.server.name=${GRAVITINO_SIMPLE_SERVER_NAME}" if [ "$JVM_VERSION" -eq 17 ]; then JAVA_OPTS+=" -XX:+IgnoreUnrecognizedVMOptions" JAVA_OPTS+=" --add-opens java.base/java.io=ALL-UNNAMED" diff --git a/bin/iceberg-rest-server.sh b/bin/iceberg-rest-server.sh new file mode 100755 index 00000000000..c4b192d1d38 --- /dev/null +++ b/bin/iceberg-rest-server.sh @@ -0,0 +1,206 @@ +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# +#set -ex +USAGE="-e Usage: bin/iceberg-rest-server.sh [--config ]\n\t + {start|run|stop|restart|status}" + +if [[ "$1" == "--config" ]]; then + shift + conf_dir="$1" + if [[ ! -d "${conf_dir}" ]]; then + echo "ERROR : ${conf_dir} is not a directory" + echo ${USAGE} + exit 1 + else + export GRAVITINO_CONF_DIR="${conf_dir}" + fi + shift +fi + +bin="$(dirname "${BASH_SOURCE-$0}")" +bin="$(cd "${bin}">/dev/null; pwd)" + +. "${bin}/common.sh" + +check_java_version + +function check_process_status() { + local pid=$(found_iceberg_rest_server_pid) + + if [[ -z "${pid}" ]]; then + echo "IcebergRESTServer is not running" + else + echo "IcebergRESTServer is running[PID:$pid]" + fi +} + +function found_iceberg_rest_server_pid() { + process_name='IcebergRESTServer'; + RUNNING_PIDS=$(ps x | grep ${process_name} | grep -v grep | awk '{print $1}'); + + if [[ -z "${RUNNING_PIDS}" ]]; then + return + fi + + if ! kill -0 ${RUNNING_PIDS} > /dev/null 2>&1; then + echo "IcebergRESTServer running but process is dead" + fi + + echo "${RUNNING_PIDS}" +} + +function wait_for_iceberg_rest_server_to_die() { + timeout=10 + timeoutTime=$(date "+%s") + let "timeoutTime+=$timeout" + currentTime=$(date "+%s") + forceKill=1 + + while [[ $currentTime -lt $timeoutTime ]]; do + local pid=$(found_iceberg_rest_server_pid) + if [[ -z "${pid}" ]]; then + forceKill=0 + break + fi + + $(kill ${pid} > /dev/null 2> /dev/null) + if kill -0 ${pid} > /dev/null 2>&1; then + sleep 3 + else + forceKill=0 + break + fi + currentTime=$(date "+%s") + done + + if [[ forceKill -ne 0 ]]; then + $(kill -9 ${pid} > /dev/null 2> /dev/null) + fi +} + +function start() { + local pid=$(found_iceberg_rest_server_pid) + + if [[ ! -z "${pid}" ]]; then + if kill -0 ${pid} >/dev/null 2>&1; then + echo "IcebergRESTServer is already running" + return 0; + fi + fi + + if [[ ! -d "${GRAVITINO_LOG_DIR}" ]]; then + echo "Log dir doesn't exist, create ${GRAVITINO_LOG_DIR}" + mkdir -p "${GRAVITINO_LOG_DIR}" + fi + + nohup ${JAVA_RUNNER} ${JAVA_OPTS} ${GRAVITINO_DEBUG_OPTS} -cp ${GRAVITINO_CLASSPATH} ${GRAVITINO_SERVER_NAME} >> "${GRAVITINO_OUTFILE}" 2>&1 & + + pid=$! + if [[ -z "${pid}" ]]; then + echo "IcebergRESTServer start error!" + return 1; + else + echo "IcebergRESTServer start success!" + fi + + sleep 2 + check_process_status +} + +function run() { + ${JAVA_RUNNER} ${JAVA_OPTS} ${GRAVITINO_DEBUG_OPTS} -cp ${GRAVITINO_CLASSPATH} ${GRAVITINO_SERVER_NAME} +} + +function stop() { + local pid + + pid=$(found_iceberg_rest_server_pid) + + if [[ -z "${pid}" ]]; then + echo "IcebergRESTServer is not running" + else + wait_for_iceberg_rest_server_to_die + echo "IcebergRESTServer stop" + fi +} + +HOSTNAME=$(hostname) +GRAVITINO_OUTFILE="${GRAVITINO_LOG_DIR}/iceberg-rest-server.out" +GRAVITINO_SERVER_NAME=com.datastrato.gravitino.server.IcebergRESTServer +GRAVITINO_SIMPLE_SERVER_NAME=iceberg-rest-server + +JAVA_OPTS+=" -Dfile.encoding=UTF-8" +JAVA_OPTS+=" -Dlog4j2.configurationFile=file://${GRAVITINO_CONF_DIR}/log4j2.properties" +JAVA_OPTS+=" -Dgravitino.log.path=${GRAVITINO_LOG_DIR} ${GRAVITINO_MEM}" +JAVA_OPTS+=" -Dgravitino.server.name=${GRAVITINO_SIMPLE_SERVER_NAME}" +if [ "$JVM_VERSION" -eq 17 ]; then + JAVA_OPTS+=" -XX:+IgnoreUnrecognizedVMOptions" + JAVA_OPTS+=" --add-opens java.base/java.io=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.lang.invoke=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.lang.reflect=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.lang=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.math=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.net=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.nio=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.text=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.time=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.util.concurrent.atomic=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.util.concurrent=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.util.regex=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/java.util=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/jdk.internal.ref=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/jdk.internal.reflect=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.sql/java.sql=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/sun.util.calendar=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/sun.nio.ch=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/sun.nio.cs=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/sun.security.action=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.base/sun.util.calendar=ALL-UNNAMED" + JAVA_OPTS+=" --add-opens java.security.jgss/sun.security.krb5=ALL-UNNAMED" +fi + +#JAVA_OPTS+=" -Djava.securit.krb5.conf=/etc/krb5.conf" + +if [ -d "${GRAVITINO_HOME}/iceberg-rest-server/libs" ]; then + addJarInDir "${GRAVITINO_HOME}/iceberg-rest-server/libs" +else + addJarInDir "${GRAVITINO_HOME}/libs" +fi + +case "${1}" in + start) + start + ;; + run) + run + ;; + stop) + stop + ;; + restart) + stop + start + ;; + status) + check_process_status + ;; + *) + echo ${USAGE} +esac diff --git a/build.gradle.kts b/build.gradle.kts index edf9bcbb5fc..3357881bbe2 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -528,7 +528,7 @@ tasks { val outputDir = projectDir.dir("distribution") val compileDistribution by registering { - dependsOn("copySubprojectDependencies", "copyCatalogLibAndConfigs", "copySubprojectLib") + dependsOn("copySubprojectDependencies", "copyCatalogLibAndConfigs", "copySubprojectLib", "iceberg:iceberg-rest-server:copyLibs") group = "gravitino distribution" outputs.dir(projectDir.dir("distribution/package")) @@ -561,8 +561,42 @@ tasks { } } + val compileIcebergRESTServer by registering { + dependsOn("iceberg:iceberg-rest-server:copyLibsToStandalonePackage") + group = "gravitino distribution" + outputs.dir(projectDir.dir("distribution/iceberg-rest-server")) + doLast { + copy { + from(projectDir.dir("conf")) { + include("iceberg-rest-server.conf.template", "log4j2.properties.template") + into("iceberg-rest-server/conf") + } + from(projectDir.dir("bin")) { + include("common.sh", "iceberg-rest-server.sh") + into("iceberg-rest-server/bin") + } + into(outputDir) + rename { fileName -> + fileName.replace(".template", "") + } + fileMode = 0b111101101 + } + + copy { + from(projectDir.dir("licenses")) { into("iceberg-rest-server/licenses") } + from(projectDir.file("LICENSE.bin")) { into("iceberg-rest-server") } + from(projectDir.file("NOTICE.bin")) { into("iceberg-rest-server") } + from(projectDir.file("README.md")) { into("iceberg-rest-server") } + into(outputDir) + rename { fileName -> + fileName.replace(".bin", "") + } + } + } + } + val assembleDistribution by registering(Tar::class) { - dependsOn("assembleTrinoConnector") + dependsOn("assembleTrinoConnector", "assembleIcebergRESTServer") group = "gravitino distribution" finalizedBy("checksumDistribution") into("${rootProject.name}-$version-bin") @@ -583,9 +617,38 @@ tasks { destinationDirectory.set(projectDir.dir("distribution")) } + val assembleIcebergRESTServer by registering(Tar::class) { + dependsOn("compileIcebergRESTServer") + // fix gradlew assembleDistribution error + mustRunAfter("compileDistribution") + group = "gravitino distribution" + finalizedBy("checksumIcebergRESTServerDistribution") + into("${rootProject.name}-iceberg-rest-server-$version-bin") + from(compileIcebergRESTServer.map { it.outputs.files.single() }) + compression = Compression.GZIP + archiveFileName.set("${rootProject.name}-iceberg-rest-server-$version-bin.tar.gz") + destinationDirectory.set(projectDir.dir("distribution")) + } + + register("checksumIcebergRESTServerDistribution") { + group = "gravitino distribution" + dependsOn(assembleIcebergRESTServer) + val archiveFile = assembleIcebergRESTServer.flatMap { it.archiveFile } + val checksumFile = archiveFile.map { archive -> + archive.asFile.let { it.resolveSibling("${it.name}.sha256") } + } + inputs.file(archiveFile) + outputs.file(checksumFile) + doLast { + checksumFile.get().writeText( + serviceOf().sha256(archiveFile.get().asFile).toString() + ) + } + } + register("checksumDistribution") { group = "gravitino distribution" - dependsOn(assembleDistribution, "checksumTrinoConnector") + dependsOn(assembleDistribution, "checksumTrinoConnector", "checksumIcebergRESTServerDistribution") val archiveFile = assembleDistribution.flatMap { it.archiveFile } val checksumFile = archiveFile.map { archive -> archive.asFile.let { it.resolveSibling("${it.name}.sha256") } @@ -623,7 +686,7 @@ tasks { register("copySubprojectDependencies", Copy::class) { subprojects.forEach() { if (!it.name.startsWith("catalog") && - !it.name.startsWith("client") && !it.name.startsWith("filesystem") && !it.name.startsWith("spark") && it.name != "trino-connector" && + !it.name.startsWith("client") && !it.name.startsWith("filesystem") && !it.name.startsWith("spark") && !it.name.startsWith("iceberg-") && it.name != "trino-connector" && it.name != "integration-test" && it.name != "bundled-catalog" && it.name != "flink-connector" ) { from(it.configurations.runtimeClasspath) @@ -638,6 +701,7 @@ tasks { !it.name.startsWith("client") && !it.name.startsWith("filesystem") && !it.name.startsWith("spark") && + !it.name.startsWith("iceberg-") && it.name != "trino-connector" && it.name != "integration-test" && it.name != "bundled-catalog" && diff --git a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts index 4a241c77465..f046be1f9d9 100644 --- a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts +++ b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts @@ -35,45 +35,18 @@ dependencies { implementation(project(":catalogs:catalog-common")) implementation(project(":common")) implementation(project(":core")) + implementation(project(":iceberg:iceberg-common")) implementation(project(":server-common")) implementation(libs.bundles.iceberg) - implementation(libs.bundles.jetty) implementation(libs.bundles.jersey) + implementation(libs.bundles.jetty) + implementation(libs.bundles.log4j) - implementation(libs.caffeine) implementation(libs.cglib) implementation(libs.commons.collections4) implementation(libs.commons.io) implementation(libs.commons.lang3) implementation(libs.guava) - implementation(libs.hive2.metastore) { - exclude("co.cask.tephra") - exclude("com.github.spotbugs") - exclude("com.google.code.findbugs", "jsr305") - exclude("com.tdunning", "json") - exclude("javax.transaction", "transaction-api") - exclude("org.apache.avro", "avro") - exclude("org.apache.hbase") - exclude("org.apache.hadoop", "hadoop-yarn-api") - exclude("org.apache.hadoop", "hadoop-yarn-server-applicationhistoryservice") - exclude("org.apache.hadoop", "hadoop-yarn-server-common") - exclude("org.apache.hadoop", "hadoop-yarn-server-resourcemanager") - exclude("org.apache.hadoop", "hadoop-yarn-server-web-proxy") - exclude("org.apache.logging.log4j") - exclude("org.apache.parquet", "parquet-hadoop-bundle") - exclude("org.apache.zookeeper") - exclude("org.eclipse.jetty.aggregate", "jetty-all") - exclude("org.eclipse.jetty.orbit", "javax.servlet") - exclude("org.pentaho") // missing dependency - exclude("org.slf4j", "slf4j-log4j12") - exclude("com.zaxxer", "HikariCP") - exclude("com.sun.jersey", "jersey-server") - } - implementation(libs.iceberg.hive.metastore) - implementation(libs.jackson.annotations) - implementation(libs.jackson.databind) - implementation(libs.jackson.datatype.jdk8) - implementation(libs.jackson.datatype.jsr310) implementation(libs.sqlite.jdbc) annotationProcessor(libs.lombok) @@ -86,13 +59,6 @@ dependencies { testImplementation(project(":server")) testImplementation(project(":server-common")) - implementation(libs.hadoop2.common) { - exclude("com.github.spotbugs") - } - implementation(libs.hadoop2.hdfs) - implementation(libs.hadoop2.mapreduce.client.core) - implementation(libs.metrics.jersey2) - testImplementation("org.scala-lang.modules:scala-collection-compat_$scalaVersion:$scalaCollectionCompatVersion") testImplementation("org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_$scalaVersion:$icebergVersion") testImplementation("org.apache.spark:spark-hive_$scalaVersion:$sparkVersion") @@ -105,6 +71,10 @@ dependencies { } testImplementation(libs.bundles.log4j) + testImplementation(libs.bundles.jersey) + testImplementation(libs.hadoop2.common) { + exclude("com.github.spotbugs") + } testImplementation(libs.jersey.test.framework.core) { exclude(group = "org.junit.jupiter") } diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java index bf520ba1a4b..67a0471f87e 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java +++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java @@ -37,7 +37,6 @@ import org.apache.gravitino.NameIdentifier; import org.apache.gravitino.Namespace; import org.apache.gravitino.SchemaChange; -import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps; import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOpsHelper; import org.apache.gravitino.connector.CatalogInfo; import org.apache.gravitino.connector.CatalogOperations; @@ -50,6 +49,9 @@ import org.apache.gravitino.exceptions.NonEmptySchemaException; import org.apache.gravitino.exceptions.SchemaAlreadyExistsException; import org.apache.gravitino.exceptions.TableAlreadyExistsException; +import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps.IcebergTableChange; import org.apache.gravitino.meta.AuditInfo; import org.apache.gravitino.rel.Column; import org.apache.gravitino.rel.Table; @@ -112,7 +114,7 @@ public void initialize( IcebergConfig icebergConfig = new IcebergConfig(resultConf); this.icebergTableOps = new IcebergTableOps(icebergConfig); - this.icebergTableOpsHelper = icebergTableOps.createIcebergTableOpsHelper(); + this.icebergTableOpsHelper = new IcebergTableOpsHelper(icebergTableOps.getCatalog()); } /** Closes the Iceberg catalog and releases the associated client pool. */ @@ -405,7 +407,7 @@ private Table internalUpdateTable(NameIdentifier tableIdent, TableChange... chan throws NoSuchTableException, IllegalArgumentException { try { String[] levels = tableIdent.namespace().levels(); - IcebergTableOpsHelper.IcebergTableChange icebergTableChange = + IcebergTableChange icebergTableChange = icebergTableOpsHelper.buildIcebergTableChanges( NameIdentifier.of(levels[levels.length - 1], tableIdent.name()), changes); LoadTableResponse loadTableResponse = icebergTableOps.updateTable(icebergTableChange); diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java index a94a1e8d3f1..abdc29bf438 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java +++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java @@ -21,16 +21,18 @@ import static org.apache.gravitino.connector.PropertyEntry.enumImmutablePropertyEntry; import static org.apache.gravitino.connector.PropertyEntry.stringRequiredPropertyEntry; +import org.apache.gravitino.iceberg.common.IcebergCatalogBackend; +import org.apache.gravitino.iceberg.common.IcebergConstants; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.AuthenticationConfig; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.KerberosConfig; import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata; import org.apache.gravitino.connector.PropertyEntry; +import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig; +import org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig; public class IcebergCatalogPropertiesMetadata extends BaseCatalogPropertiesMetadata { public static final String CATALOG_BACKEND = IcebergConstants.CATALOG_BACKEND; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOpsHelper.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOpsHelper.java index eef1457d005..176e2cbbeb3 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOpsHelper.java +++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOpsHelper.java @@ -27,11 +27,9 @@ import java.util.Arrays; import java.util.List; import java.util.Set; -import javax.ws.rs.NotSupportedException; -import lombok.Getter; -import lombok.Setter; import org.apache.gravitino.NameIdentifier; import org.apache.gravitino.catalog.lakehouse.iceberg.converter.IcebergDataTypeConverter; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps.IcebergTableChange; import org.apache.gravitino.rel.TableChange; import org.apache.gravitino.rel.TableChange.AddColumn; import org.apache.gravitino.rel.TableChange.After; @@ -59,7 +57,6 @@ import org.apache.iceberg.types.Types.StructType; public class IcebergTableOpsHelper { - @VisibleForTesting public static final Joiner DOT = Joiner.on("."); private static final Set IcebergReservedProperties = ImmutableSet.of( @@ -76,18 +73,6 @@ public IcebergTableOpsHelper(Catalog icebergCatalog) { this.icebergCatalog = icebergCatalog; } - @Getter - @Setter - public static final class IcebergTableChange { - private TableIdentifier tableIdentifier; - private Transaction transaction; - - IcebergTableChange(TableIdentifier tableIdentifier, Transaction transaction) { - this.tableIdentifier = tableIdentifier; - this.transaction = transaction; - } - } - private void doDeleteColumn( UpdateSchema icebergUpdateSchema, DeleteColumn deleteColumn, Schema icebergTableSchema) { NestedField deleteField = icebergTableSchema.findField(DOT.join(deleteColumn.fieldName())); @@ -142,7 +127,7 @@ private void doMoveColumn( } else if (columnPosition instanceof TableChange.First) { icebergUpdateSchema.moveFirst(DOT.join(fieldName)); } else { - throw new NotSupportedException( + throw new UnsupportedOperationException( "Iceberg doesn't support column position: " + columnPosition.getClass().getSimpleName()); } } @@ -237,7 +222,7 @@ private void alterTableProperty( } else if (change instanceof SetProperty) { doSetProperty(icebergUpdateProperties, (SetProperty) change); } else { - throw new NotSupportedException( + throw new UnsupportedOperationException( "Iceberg doesn't support table change: " + change.getClass().getSimpleName() + " for now"); @@ -269,7 +254,7 @@ private void alterTableColumn( } else if (change instanceof TableChange.UpdateColumnAutoIncrement) { throw new IllegalArgumentException("Iceberg doesn't support auto increment column"); } else { - throw new NotSupportedException( + throw new UnsupportedOperationException( "Iceberg doesn't support " + change.getClass().getSimpleName() + " for now"); } } @@ -307,7 +292,8 @@ public IcebergTableChange buildIcebergTableChanges( } else if (change instanceof RenameTable) { throw new RuntimeException("RenameTable shouldn't use tableUpdate interface"); } else { - throw new NotSupportedException("Iceberg doesn't support " + change.getClass() + "for now"); + throw new UnsupportedOperationException( + "Iceberg doesn't support " + change.getClass() + "for now"); } } diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider b/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider index 171e2ad8c91..2e92a61a120 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider +++ b/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.CatalogProvider @@ -16,4 +16,4 @@ # specific language governing permissions and limitations # under the License. # -org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalog \ No newline at end of file +org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalog diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergCatalog.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergCatalog.java index 3fa4a8acc86..f00807c39e3 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergCatalog.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergCatalog.java @@ -27,10 +27,10 @@ import java.util.Map; import org.apache.gravitino.Namespace; import org.apache.gravitino.catalog.PropertiesMetadataHelpers; -import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps; import org.apache.gravitino.connector.CatalogOperations; import org.apache.gravitino.connector.HasPropertyMetadata; import org.apache.gravitino.connector.PropertiesMetadata; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps; import org.apache.gravitino.meta.AuditInfo; import org.apache.gravitino.meta.CatalogEntity; import org.apache.iceberg.rest.responses.ListNamespacesResponse; diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergBaseIT.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergBaseIT.java index 8ad6ffe59be..078fe803cd6 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergBaseIT.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergBaseIT.java @@ -46,16 +46,16 @@ import org.apache.gravitino.SchemaChange; import org.apache.gravitino.SupportsSchemas; import org.apache.gravitino.auth.AuthConstants; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergSchemaPropertiesMetadata; import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergTable; import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOpsHelper; -import org.apache.gravitino.catalog.lakehouse.iceberg.utils.IcebergCatalogUtil; import org.apache.gravitino.client.GravitinoMetalake; import org.apache.gravitino.dto.util.DTOConverters; import org.apache.gravitino.exceptions.NoSuchSchemaException; import org.apache.gravitino.exceptions.SchemaAlreadyExistsException; import org.apache.gravitino.exceptions.TableAlreadyExistsException; +import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.common.utils.IcebergCatalogUtil; import org.apache.gravitino.integration.test.container.ContainerSuite; import org.apache.gravitino.integration.test.util.AbstractIT; import org.apache.gravitino.integration.test.util.GravitinoITUtils; diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java index bdff4da228e..cd220546ae6 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java @@ -18,12 +18,14 @@ */ package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test; -import static org.apache.gravitino.catalog.lakehouse.iceberg.authentication.AuthenticationConfig.AUTH_TYPE_KEY; -import static org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.KerberosConfig.IMPERSONATION_ENABLE_KEY; -import static org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.KerberosConfig.KET_TAB_URI_KEY; -import static org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.KerberosConfig.PRINCIPAL_KEY; import static org.apache.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX; + +import static org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig.AUTH_TYPE_KEY; +import static org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig.IMPERSONATION_ENABLE_KEY; +import static org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig.KET_TAB_URI_KEY; +import static org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig.PRINCIPAL_KEY; + import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; @@ -36,10 +38,10 @@ import org.apache.commons.io.FileUtils; import org.apache.gravitino.Catalog; import org.apache.gravitino.NameIdentifier; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; import org.apache.gravitino.client.GravitinoAdminClient; import org.apache.gravitino.client.GravitinoMetalake; import org.apache.gravitino.client.KerberosTokenProvider; +import org.apache.gravitino.iceberg.common.IcebergConfig; import org.apache.gravitino.integration.test.container.ContainerSuite; import org.apache.gravitino.integration.test.container.HiveContainer; import org.apache.gravitino.integration.test.util.AbstractIT; diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java index 73eec2fb5ab..a7f6efd6a07 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java @@ -18,13 +18,11 @@ */ package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test; +import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.common.IcebergConstants; import java.util.Map; -import org.apache.gravitino.auxiliary.AuxiliaryServiceManager; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService; import org.apache.gravitino.integration.test.container.HiveContainer; import org.apache.gravitino.server.web.JettyServerConfig; -import org.apache.gravitino.utils.MapUtils; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.TestInstance; @@ -35,8 +33,8 @@ public class CatalogIcebergRestIT extends CatalogIcebergBaseIT { @Override protected void initIcebergCatalogProperties() { Map map = - serverConfig.getConfigsWithPrefix(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX); - map = MapUtils.getPrefixMap(map, IcebergRESTService.SERVICE_NAME + "."); + serverConfig.getConfigsWithPrefix( + String.format("gravitino.%s.", IcebergConstants.GRAVITINO_ICEBERG_REST_SERVICE_NAME)); IcebergConfig icebergConfig = new IcebergConfig(map); String host = icebergConfig.get(JettyServerConfig.WEBSERVER_HOST); int port = icebergConfig.get(JettyServerConfig.WEBSERVER_HTTP_PORT); diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/TestMultipleJDBCLoad.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/TestMultipleJDBCLoad.java index 605e97553f3..5f78ab57377 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/TestMultipleJDBCLoad.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/TestMultipleJDBCLoad.java @@ -30,8 +30,8 @@ import java.util.Map; import org.apache.gravitino.Catalog; import org.apache.gravitino.NameIdentifier; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; import org.apache.gravitino.client.GravitinoMetalake; +import org.apache.gravitino.iceberg.common.IcebergConfig; import org.apache.gravitino.integration.test.container.MySQLContainer; import org.apache.gravitino.integration.test.container.PostgreSQLContainer; import org.apache.gravitino.integration.test.util.AbstractIT; diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/TestIcebergTableUpdate.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/TestIcebergTableUpdate.java index 96f826490e5..f8d771da2df 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/TestIcebergTableUpdate.java +++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/TestIcebergTableUpdate.java @@ -21,7 +21,8 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.gravitino.NameIdentifier; -import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOpsHelper.IcebergTableChange; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps; +import org.apache.gravitino.iceberg.common.ops.IcebergTableOps.IcebergTableChange; import org.apache.gravitino.rel.TableChange; import org.apache.gravitino.rel.TableChange.ColumnPosition; import org.apache.gravitino.rel.types.Types; @@ -79,7 +80,7 @@ public class TestIcebergTableUpdate { @BeforeEach public void init() { icebergTableOps = new IcebergTableOps(); - icebergTableOpsHelper = icebergTableOps.createIcebergTableOpsHelper(); + icebergTableOpsHelper = new IcebergTableOpsHelper(icebergTableOps.getCatalog()); createNamespace(TEST_NAMESPACE_NAME); createTable(TEST_NAMESPACE_NAME, TEST_TABLE_NAME); } diff --git a/conf/gravitino.conf.template b/conf/gravitino.conf.template index 22fe9241553..dab4d86cc92 100644 --- a/conf/gravitino.conf.template +++ b/conf/gravitino.conf.template @@ -63,9 +63,12 @@ gravitino.catalog.cache.evictionIntervalMs = 3600000 # Auxiliary service names, separate by ',' gravitino.auxService.names = iceberg-rest # Iceberg REST service classpath -gravitino.auxService.iceberg-rest.classpath = catalogs/lakehouse-iceberg/libs, catalogs/lakehouse-iceberg/conf +gravitino.iceberg-rest.classpath = iceberg-rest-server/libs # Iceberg REST service host -gravitino.auxService.iceberg-rest.host = 0.0.0.0 +gravitino.iceberg-rest.host = 0.0.0.0 # Iceberg REST service http port -gravitino.auxService.iceberg-rest.httpPort = 9001 - +gravitino.iceberg-rest.httpPort = 9001 +# The backend Iceberg catalog for Iceberg REST service, it's recommanded to change to hive or jdbc +gravitino.iceberg-rest.catalog-backend = memory +# The warehouse directory of Iceberg catalog for Iceberg REST service +gravitino.iceberg-rest.warehouse = /tmp/ diff --git a/conf/iceberg-rest-server.conf.template b/conf/iceberg-rest-server.conf.template new file mode 100644 index 00000000000..b4cfddab180 --- /dev/null +++ b/conf/iceberg-rest-server.conf.template @@ -0,0 +1,47 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# THE CONFIGURATION FOR Iceberg REST SERVER +gravitino.iceberg-rest.shutdown.timeout = 3000 + +# THE CONFIGURATION FOR Iceberg REST WEB SERVER +# The host name of the built-in web server +gravitino.iceberg-rest.host = 0.0.0.0 +# The http port number of the built-in web server +gravitino.iceberg-rest.httpPort = 9001 +# The min thread size of the built-in web server +gravitino.iceberg-rest.minThreads = 24 +# The max thread size of the built-in web server +gravitino.iceberg-rest.maxThreads = 200 +# The stop timeout of the built-in web server +gravitino.iceberg-rest.stopTimeout = 30000 +# The timeout of idle connections +gravitino.iceberg-rest.idleTimeout = 30000 +# The executor thread pool work queue size of the built-in web server +gravitino.iceberg-rest.threadPoolWorkQueueSize = 100 +# The request header size of the built-in web server +gravitino.iceberg-rest.requestHeaderSize = 131072 +# The response header size of the built-in web server +gravitino.iceberg-rest.responseHeaderSize = 131072 + +# THE CONFIGURATION FOR Iceberg backend catalog +# The backend Iceberg catalog, it's recommanded to change to hive or jdbc +gravitino.iceberg-rest.catalog-backend = memory +# The warehouse directory of Iceberg catalog +gravitino.iceberg-rest.warehouse = /tmp/ diff --git a/conf/log4j2.properties.template b/conf/log4j2.properties.template index 9931e9c1b20..bd26f33e37f 100644 --- a/conf/log4j2.properties.template +++ b/conf/log4j2.properties.template @@ -21,12 +21,13 @@ status = warn # Log files location property.basePath = ${sys:gravitino.log.path} +property.serverName = ${sys:gravitino.server.name} # RollingFileAppender name, pattern, path and rollover policy appender.rolling.type = RollingFile appender.rolling.name = fileLogger -appender.rolling.fileName = ${basePath}/gravitino-server.log -appender.rolling.filePattern = ${basePath}/gravitino-server_%d{yyyyMMdd}.log.gz +appender.rolling.fileName = ${basePath}/${serverName}.log +appender.rolling.filePattern = ${basePath}/${serverName}_%d{yyyyMMdd}.log.gz appender.rolling.layout.type = PatternLayout appender.rolling.layout.pattern = %d{yyyy-MM-dd HH:mm:ss.SSS} %level [%t] [%l] - %msg%n appender.rolling.policies.type = Policies diff --git a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java index cf95dd7e74c..5393309277e 100644 --- a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java +++ b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java @@ -120,13 +120,22 @@ public static GravitinoEnv getInstance() { * * @param config The configuration object to initialize the environment. */ - public void initialize(Config config) { + public void initialize(Config config, boolean initGravitinoServerComponent) { LOG.info("Initializing Gravitino Environment..."); this.config = config; this.metricsSystem = new MetricsSystem(); metricsSystem.register(new JVMMetricsSource()); + this.eventListenerManager = new EventListenerManager(); + eventListenerManager.init( + config.getConfigsWithPrefix(EventListenerManager.GRAVITINO_EVENT_LISTENER_PREFIX)); + EventBus eventBus = eventListenerManager.createEventBus(); + + if (!initGravitinoServerComponent) { + return; + } + // Initialize EntityStore this.entityStore = EntityStoreFactory.createEntityStore(config); entityStore.initialize(config); @@ -134,11 +143,6 @@ public void initialize(Config config) { // create and initialize a random id generator this.idGenerator = new RandomIdGenerator(); - this.eventListenerManager = new EventListenerManager(); - eventListenerManager.init( - config.getConfigsWithPrefix(EventListenerManager.GRAVITINO_EVENT_LISTENER_PREFIX)); - EventBus eventBus = eventListenerManager.createEventBus(); - // Create and initialize metalake related modules MetalakeManager metalakeManager = new MetalakeManager(entityStore, idGenerator); MetalakeNormalizeDispatcher metalakeNormalizeDispatcher = @@ -190,8 +194,7 @@ public void initialize(Config config) { } this.auxServiceManager = new AuxiliaryServiceManager(); - this.auxServiceManager.serviceInit( - config.getConfigsWithPrefix(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX)); + this.auxServiceManager.serviceInit(config); // Tree lock this.lockManager = new LockManager(config); diff --git a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java index 45012eead74..37ebcf05085 100644 --- a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java +++ b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java @@ -35,6 +35,7 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; +import org.apache.gravitino.Config; import org.apache.gravitino.utils.IsolatedClassLoader; import org.apache.gravitino.utils.MapUtils; import org.slf4j.Logger; @@ -124,7 +125,7 @@ private void registerAuxService(String auxServiceName, Map confi StringUtils.isNoneBlank(classpath), String.format( "AuxService:%s, %s%s.%s is not set in configuration", - auxServiceName, GRAVITINO_AUX_SERVICE_PREFIX, auxServiceName, AUX_SERVICE_CLASSPATH)); + auxServiceName, "gravitino.", auxServiceName, AUX_SERVICE_CLASSPATH)); List validPaths = splitter @@ -180,15 +181,16 @@ private void doWithClassLoader(String auxServiceName, Consumer config) { - registerAuxServices(config); + public void serviceInit(Config gravitinoConfig) { + Map serviceConfigs = extractAuxiliaryServiceConfigs(gravitinoConfig); + registerAuxServices(serviceConfigs); auxServices.forEach( (auxServiceName, auxService) -> { doWithClassLoader( auxServiceName, cl -> auxService.serviceInit( - MapUtils.getPrefixMap(config, DOT.join(auxServiceName, "")))); + MapUtils.getPrefixMap(serviceConfigs, DOT.join(auxServiceName, "")))); }); } @@ -219,4 +221,31 @@ public void serviceStop() throws Exception { throw firstException; } } + + // Extract aux service configs, transform gravitino.$serviceName.key to $serviceName.key. + @VisibleForTesting + static Map extractAuxiliaryServiceConfigs(Config config) { + String auxServiceNames = + config + .getConfigsWithPrefix(GRAVITINO_AUX_SERVICE_PREFIX) + .getOrDefault(AUX_SERVICE_NAMES, ""); + Map serviceConfigs = new HashMap<>(); + serviceConfigs.put(AUX_SERVICE_NAMES, auxServiceNames); + splitter + .omitEmptyStrings() + .trimResults() + .splitToStream(auxServiceNames) + .forEach( + name -> + config + .getAllConfig() + .forEach( + (k, v) -> { + String prefix = "gravitino." + name + "."; + if (k.startsWith(prefix)) { + serviceConfigs.put(k.substring("gravitino.".length()), v); + } + })); + return serviceConfigs; + } } diff --git a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java index 2806a1dcd31..9582d1b8de5 100644 --- a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java +++ b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java @@ -29,16 +29,31 @@ import com.google.common.collect.ImmutableMap; import java.util.Collections; +import org.apache.gravitino.Config; import org.apache.gravitino.utils.IsolatedClassLoader; +import java.util.Map; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; public class TestAuxiliaryServiceManager { + static class DummyConfig extends Config { + public static DummyConfig of(Map m) { + DummyConfig dummyConfig = new DummyConfig(); + dummyConfig.loadFromMap(m, k -> true); + return dummyConfig; + } + } + @Test public void testGravitinoAuxServiceManagerEmptyServiceName() throws Exception { AuxiliaryServiceManager auxServiceManager = new AuxiliaryServiceManager(); - auxServiceManager.serviceInit(ImmutableMap.of(AuxiliaryServiceManager.AUX_SERVICE_NAMES, "")); + auxServiceManager.serviceInit( + DummyConfig.of( + ImmutableMap.of( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + ""))); auxServiceManager.serviceStart(); auxServiceManager.serviceStop(); } @@ -50,7 +65,11 @@ public void testGravitinoAuxServiceNotSetClassPath() { IllegalArgumentException.class, () -> auxServiceManager.serviceInit( - ImmutableMap.of(AuxiliaryServiceManager.AUX_SERVICE_NAMES, "mock1"))); + DummyConfig.of( + ImmutableMap.of( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + "mock1")))); } @Test @@ -70,13 +89,15 @@ public void testGravitinoAuxServiceManager() throws Exception { doReturn(auxService2).when(spyAuxManager).loadAuxService("mock2", isolatedClassLoader); spyAuxManager.serviceInit( - ImmutableMap.of( - AuxiliaryServiceManager.AUX_SERVICE_NAMES, - "mock1,mock2", - "mock1." + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH, - "/tmp", - "mock2." + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH, - "/tmp")); + DummyConfig.of( + ImmutableMap.of( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + "mock1,mock2", + "gravitino.mock1." + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH, + "/tmp", + "gravitino.mock2." + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH, + "/tmp"))); verify(auxService, times(1)).serviceInit(any()); verify(auxService2, times(1)).serviceInit(any()); @@ -88,4 +109,32 @@ public void testGravitinoAuxServiceManager() throws Exception { verify(auxService, times(1)).serviceStop(); verify(auxService2, times(1)).serviceStop(); } + + @Test + void testAuxiliaryServiceConfigs() { + Map m = + ImmutableMap.of( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + "a,b", + "gravitino.a.test1", + "test1", + "gravitino.b.test2", + "test2", + "gravitino.aa.test3", + "test3"); + DummyConfig config = new DummyConfig(); + config.loadFromMap(m, k -> true); + Map serviceConfigs = + AuxiliaryServiceManager.extractAuxiliaryServiceConfigs(config); + Assertions.assertEquals( + ImmutableMap.of( + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + "a,b", + "a.test1", + "test1", + "b.test2", + "test2"), + serviceConfigs); + } } diff --git a/iceberg/iceberg-common/build.gradle.kts b/iceberg/iceberg-common/build.gradle.kts new file mode 100644 index 00000000000..415d858b88a --- /dev/null +++ b/iceberg/iceberg-common/build.gradle.kts @@ -0,0 +1,87 @@ +/* + * 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. + */ +description = "iceberg-common" + +plugins { + `maven-publish` + id("java") + id("idea") +} + +dependencies { + implementation(project(":core")) + implementation(project(":common")) + implementation(project(":server-common")) + implementation(libs.bundles.iceberg) + implementation(libs.bundles.log4j) + implementation(libs.caffeine) + implementation(libs.commons.lang3) + implementation(libs.guava) + implementation(libs.iceberg.hive.metastore) + implementation(libs.hadoop2.common) { + exclude("com.github.spotbugs") + exclude("com.sun.jersey") + exclude("javax.servlet") + exclude("org.mortbay.jetty") + } + implementation(libs.hadoop2.hdfs) { + exclude("com.sun.jersey") + exclude("javax.servlet") + exclude("org.mortbay.jetty") + } + implementation(libs.hadoop2.mapreduce.client.core) { + exclude("com.sun.jersey") + exclude("javax.servlet") + exclude("org.mortbay.jetty") + } + implementation(libs.hive2.metastore) { + exclude("co.cask.tephra") + exclude("com.github.spotbugs") + exclude("com.google.code.findbugs", "jsr305") + exclude("com.sun.jersey") + exclude("com.tdunning", "json") + exclude("com.zaxxer", "HikariCP") + exclude("javax.servlet") + exclude("javax.transaction", "transaction-api") + exclude("org.apache.avro", "avro") + exclude("org.apache.hbase") + exclude("org.apache.hadoop", "hadoop-yarn-api") + exclude("org.apache.hadoop", "hadoop-yarn-server-applicationhistoryservice") + exclude("org.apache.hadoop", "hadoop-yarn-server-common") + exclude("org.apache.hadoop", "hadoop-yarn-server-resourcemanager") + exclude("org.apache.hadoop", "hadoop-yarn-server-web-proxy") + exclude("org.apache.logging.log4j") + exclude("org.apache.parquet", "parquet-hadoop-bundle") + exclude("org.apache.zookeeper") + exclude("org.eclipse.jetty.aggregate", "jetty-all") + exclude("org.eclipse.jetty.orbit", "javax.servlet") + exclude("org.mortbay.jetty") + exclude("org.pentaho") // missing dependency + exclude("org.slf4j", "slf4j-log4j12") + } + + annotationProcessor(libs.lombok) + compileOnly(libs.lombok) + + testImplementation(libs.junit.jupiter.api) + testImplementation(libs.junit.jupiter.params) + testImplementation(libs.sqlite.jdbc) + + testRuntimeOnly(libs.junit.jupiter.engine) +} diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/catalog/ClosableHiveCatalog.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ClosableHiveCatalog.java similarity index 96% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/catalog/ClosableHiveCatalog.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ClosableHiveCatalog.java index 863f9b57335..a107ddb3392 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/catalog/ClosableHiveCatalog.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ClosableHiveCatalog.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.catalog; +package org.apache.gravitino.iceberg.common; import java.io.Closeable; import java.io.IOException; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogBackend.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergCatalogBackend.java similarity index 94% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogBackend.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergCatalogBackend.java index 4e9d92c4356..63fb07605c9 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogBackend.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergCatalogBackend.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg; +package org.apache.gravitino.iceberg.common; public enum IcebergCatalogBackend { HIVE, diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java similarity index 84% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java index dc0a9672a22..657c0cc9a46 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConfig.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java @@ -17,14 +17,13 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg; +package org.apache.gravitino.iceberg.common; import com.google.common.collect.ImmutableMap; import java.util.Map; import java.util.Optional; import org.apache.commons.lang3.StringUtils; import org.apache.gravitino.Config; -import org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics.IcebergMetricsManager; import org.apache.gravitino.config.ConfigBuilder; import org.apache.gravitino.config.ConfigConstants; import org.apache.gravitino.config.ConfigEntry; @@ -32,16 +31,17 @@ import org.apache.gravitino.server.web.OverwriteDefaultConfig; public class IcebergConfig extends Config implements OverwriteDefaultConfig { + public static final String ICEBERG_CONFIG_PREFIX = "gravitino.iceberg-rest."; public static final ConfigEntry CATALOG_BACKEND = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.CATALOG_BACKEND) + new ConfigBuilder(IcebergConstants.CATALOG_BACKEND) .doc("Catalog backend of Gravitino Iceberg catalog") .version(ConfigConstants.VERSION_0_2_0) .stringConf() .createWithDefault("memory"); public static final ConfigEntry CATALOG_WAREHOUSE = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.WAREHOUSE) + new ConfigBuilder(IcebergConstants.WAREHOUSE) .doc("Warehouse directory of catalog") .version(ConfigConstants.VERSION_0_2_0) .stringConf() @@ -49,7 +49,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .create(); public static final ConfigEntry CATALOG_URI = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.URI) + new ConfigBuilder(IcebergConstants.URI) .doc("The uri config of the Iceberg catalog") .version(ConfigConstants.VERSION_0_2_0) .stringConf() @@ -57,7 +57,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .create(); public static final ConfigEntry JDBC_USER = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_USER) + new ConfigBuilder(IcebergConstants.GRAVITINO_JDBC_USER) .doc("The username of the Jdbc connection") .version(ConfigConstants.VERSION_0_2_0) .stringConf() @@ -65,7 +65,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .create(); public static final ConfigEntry JDBC_PASSWORD = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_PASSWORD) + new ConfigBuilder(IcebergConstants.GRAVITINO_JDBC_PASSWORD) .doc("The password of the Jdbc connection") .version(ConfigConstants.VERSION_0_2_0) .stringConf() @@ -73,7 +73,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .create(); public static final ConfigEntry JDBC_DRIVER = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.GRAVITINO_JDBC_DRIVER) + new ConfigBuilder(IcebergConstants.GRAVITINO_JDBC_DRIVER) .doc("The driver of the Jdbc connection") .version(ConfigConstants.VERSION_0_3_0) .stringConf() @@ -81,21 +81,21 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .create(); public static final ConfigEntry JDBC_INIT_TABLES = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_INITIALIZE) + new ConfigBuilder(IcebergConstants.ICEBERG_JDBC_INITIALIZE) .doc("Whether to initialize meta tables when create Jdbc catalog") .version(ConfigConstants.VERSION_0_2_0) .booleanConf() .createWithDefault(true); public static final ConfigEntry ICEBERG_METRICS_STORE = - new ConfigBuilder(IcebergMetricsManager.ICEBERG_METRICS_STORE) + new ConfigBuilder(IcebergConstants.ICEBERG_METRICS_STORE) .doc("The store to save Iceberg metrics") .version(ConfigConstants.VERSION_0_4_0) .stringConf() .create(); public static final ConfigEntry ICEBERG_METRICS_STORE_RETAIN_DAYS = - new ConfigBuilder(IcebergMetricsManager.ICEBERG_METRICS_STORE_RETAIN_DAYS) + new ConfigBuilder(IcebergConstants.ICEBERG_METRICS_STORE_RETAIN_DAYS) .doc( "The retain days of Iceberg metrics, the value not greater than 0 means retain forever") .version(ConfigConstants.VERSION_0_4_0) @@ -103,7 +103,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .createWithDefault(-1); public static final ConfigEntry ICEBERG_METRICS_QUEUE_CAPACITY = - new ConfigBuilder(IcebergMetricsManager.ICEBERG_METRICS_QUEUE_CAPACITY) + new ConfigBuilder(IcebergConstants.ICEBERG_METRICS_QUEUE_CAPACITY) .doc("The capacity for Iceberg metrics queues, should greater than 0") .version(ConfigConstants.VERSION_0_4_0) .intConf() @@ -111,7 +111,7 @@ public class IcebergConfig extends Config implements OverwriteDefaultConfig { .createWithDefault(1000); public static final ConfigEntry CATALOG_BACKEND_NAME = - new ConfigBuilder(IcebergCatalogPropertiesMetadata.CATALOG_BACKEND_NAME) + new ConfigBuilder(IcebergConstants.CATALOG_BACKEND_NAME) .doc("The catalog name for Iceberg catalog backend") .version(ConfigConstants.VERSION_0_5_2) .stringConf() diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java new file mode 100644 index 00000000000..b3a3e738be8 --- /dev/null +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.iceberg.common; + +public class IcebergConstants { + public static final String CATALOG_BACKEND = "catalog-backend"; + + public static final String GRAVITINO_JDBC_USER = "jdbc-user"; + public static final String ICEBERG_JDBC_USER = "jdbc.user"; + + public static final String GRAVITINO_JDBC_PASSWORD = "jdbc-password"; + public static final String ICEBERG_JDBC_PASSWORD = "jdbc.password"; + public static final String ICEBERG_JDBC_INITIALIZE = "jdbc-initialize"; + + public static final String GRAVITINO_JDBC_DRIVER = "jdbc-driver"; + public static final String WAREHOUSE = "warehouse"; + public static final String URI = "uri"; + public static final String CATALOG_BACKEND_NAME = "catalog-backend-name"; + + public static final String ICEBERG_METRICS_STORE = "metricsStore"; + public static final String ICEBERG_METRICS_STORE_RETAIN_DAYS = "metricsStoreRetainDays"; + public static final String ICEBERG_METRICS_QUEUE_CAPACITY = "metricsQueueCapacity"; + + public static final String GRAVITINO_ICEBERG_REST_SERVICE_NAME = "iceberg-rest"; +} diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/AuthenticationConfig.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/AuthenticationConfig.java similarity index 98% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/AuthenticationConfig.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/AuthenticationConfig.java index d4050c3d17a..7c791fe5795 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/AuthenticationConfig.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/AuthenticationConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.authentication; +package org.apache.gravitino.iceberg.common.authentication; import com.google.common.collect.ImmutableMap; import java.util.Map; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/FetchFileUtils.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/FetchFileUtils.java similarity index 96% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/FetchFileUtils.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/FetchFileUtils.java index 17a0f3efd8a..96d91765a25 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/FetchFileUtils.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/FetchFileUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos; +package org.apache.gravitino.iceberg.common.authentication.kerberos; import java.io.File; import java.io.IOException; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/HiveBackendProxy.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java similarity index 94% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/HiveBackendProxy.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java index 2999b22acd0..45bec6488de 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/HiveBackendProxy.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java @@ -17,8 +17,7 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos; - +package org.apache.gravitino.iceberg.common.authentication.kerberos; import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -27,7 +26,7 @@ import net.sf.cglib.proxy.Enhancer; import net.sf.cglib.proxy.MethodInterceptor; import net.sf.cglib.proxy.MethodProxy; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergHiveCachedClientPool; +import org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool; import org.apache.gravitino.utils.PrincipalUtils; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.thrift.DelegationTokenIdentifier; @@ -57,7 +56,7 @@ public HiveBackendProxy( try { proxyUser = UserGroupInformation.getCurrentUser(); - // Replace the original client pool with IcebergHiveCachedClientPool. Why do we need to do + // Replace the original client pool with org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool. Why do we need to do // this? Because the original client pool in Iceberg uses a fixed username to create the // client pool, and it will not work with kerberos authentication. We need to create a new // client pool with the current user. For more, please see CachedClientPool#clientPool and diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosClient.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosClient.java similarity index 98% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosClient.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosClient.java index d717906ff26..0fb64d9be3b 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosClient.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosClient.java @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos; +package org.apache.gravitino.iceberg.common.authentication.kerberos; import com.google.common.base.Preconditions; import com.google.common.base.Splitter; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosConfig.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java similarity index 96% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosConfig.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java index b1a43944e56..6f3fa9c0214 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/authentication/kerberos/KerberosConfig.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java @@ -17,16 +17,15 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos; - +package org.apache.gravitino.iceberg.common.authentication.kerberos; import com.google.common.collect.ImmutableMap; import java.util.Map; import org.apache.commons.lang3.StringUtils; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.AuthenticationConfig; import org.apache.gravitino.config.ConfigBuilder; import org.apache.gravitino.config.ConfigConstants; import org.apache.gravitino.config.ConfigEntry; import org.apache.gravitino.connector.PropertyEntry; +import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig; public class KerberosConfig extends AuthenticationConfig { public static final String KET_TAB_URI_KEY = "authentication.kerberos.keytab-uri"; diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOps.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java similarity index 91% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOps.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java index d99e5f906f9..6f51d813934 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/ops/IcebergTableOps.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java @@ -16,19 +16,19 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.ops; +package org.apache.gravitino.iceberg.common.ops; import com.google.common.base.Preconditions; import java.sql.Driver; import java.sql.DriverManager; import java.util.Collections; import java.util.Optional; -import javax.ws.rs.NotSupportedException; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; -import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOpsHelper.IcebergTableChange; -import org.apache.gravitino.catalog.lakehouse.iceberg.utils.IcebergCatalogUtil; +import org.apache.gravitino.iceberg.common.IcebergCatalogBackend; +import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.common.utils.IcebergCatalogUtil; import org.apache.gravitino.utils.IsolatedClassLoader; +import lombok.Getter; +import lombok.Setter; import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -52,7 +52,7 @@ public class IcebergTableOps implements AutoCloseable { public static final Logger LOG = LoggerFactory.getLogger(IcebergTableOps.class); - protected Catalog catalog; + @Getter protected Catalog catalog; private SupportsNamespaces asNamespaceCatalog; private final String catalogType; private String catalogUri = null; @@ -73,17 +73,14 @@ public IcebergTableOps() { this(new IcebergConfig(Collections.emptyMap())); } - public IcebergTableOpsHelper createIcebergTableOpsHelper() { - return new IcebergTableOpsHelper(catalog); - } - private void validateNamespace(Optional namespace) { namespace.ifPresent( n -> Preconditions.checkArgument( n.toString().isEmpty() == false, "Namespace couldn't be empty")); if (asNamespaceCatalog == null) { - throw new NotSupportedException("The underlying catalog doesn't support namespace operation"); + throw new UnsupportedOperationException( + "The underlying catalog doesn't support namespace operation"); } } @@ -214,4 +211,17 @@ private void closeDriverLoadedByIsolatedClassLoader(String uri) { private void closePostgreSQLCatalogResource() { closeDriverLoadedByIsolatedClassLoader(catalogUri); } + + @Getter + @Setter + public static final class IcebergTableChange { + + private TableIdentifier tableIdentifier; + private Transaction transaction; + + public IcebergTableChange(TableIdentifier tableIdentifier, Transaction transaction) { + this.tableIdentifier = tableIdentifier; + this.transaction = transaction; + } + } } diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/IcebergCatalogUtil.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java similarity index 84% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/IcebergCatalogUtil.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java index 4860ee1b3a4..ebceaf391bf 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/IcebergCatalogUtil.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java @@ -16,23 +16,21 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.utils; - -import static org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_INITIALIZE; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; +package org.apache.gravitino.iceberg.common.utils; +import org.apache.gravitino.iceberg.common.ClosableHiveCatalog; +import org.apache.gravitino.iceberg.common.IcebergConstants; +import com.google.common.base.Preconditions; import java.io.File; import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend; -import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.AuthenticationConfig; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.HiveBackendProxy; -import org.apache.gravitino.catalog.lakehouse.iceberg.authentication.kerberos.KerberosClient; -import org.apache.gravitino.catalog.lakehouse.iceberg.catalog.ClosableHiveCatalog; +import org.apache.gravitino.iceberg.common.IcebergCatalogBackend; +import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig; +import org.apache.gravitino.iceberg.common.authentication.kerberos.HiveBackendProxy; +import org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosClient; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.iceberg.CatalogProperties; @@ -73,8 +71,8 @@ private static HiveCatalog loadHiveCatalog(Map properties) { } else if (authenticationConfig.isKerberosAuth()) { Map resultProperties = new HashMap<>(properties); resultProperties.put(CatalogProperties.CLIENT_POOL_CACHE_KEYS, "USER_NAME"); - hdfsConfiguration.set(HADOOP_SECURITY_AUTHORIZATION, "true"); - hdfsConfiguration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); + // hdfsConfiguration.set(HADOOP_SECURITY_AUTHORIZATION, "true"); + // hdfsConfiguration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos"); hiveCatalog.setConf(hdfsConfiguration); hiveCatalog.initialize(icebergCatalogName, properties); @@ -111,8 +109,12 @@ private static JdbcCatalog loadJdbcCatalog(Map properties) { String driverClassName = icebergConfig.getJdbcDriver(); String icebergCatalogName = icebergConfig.getCatalogBackendName("jdbc"); - icebergConfig.get(IcebergConfig.JDBC_USER); - icebergConfig.get(IcebergConfig.JDBC_PASSWORD); + Preconditions.checkNotNull( + properties.get(IcebergConstants.ICEBERG_JDBC_USER), + IcebergConstants.ICEBERG_JDBC_USER + " is null"); + Preconditions.checkNotNull( + properties.get(IcebergConstants.ICEBERG_JDBC_PASSWORD), + IcebergConstants.ICEBERG_JDBC_PASSWORD + " is null"); try { // Load the jdbc driver @@ -124,7 +126,8 @@ private static JdbcCatalog loadJdbcCatalog(Map properties) { new JdbcCatalog( null, null, - Boolean.parseBoolean(properties.getOrDefault(ICEBERG_JDBC_INITIALIZE, "true"))); + Boolean.parseBoolean( + properties.getOrDefault(IcebergConstants.ICEBERG_JDBC_INITIALIZE, "true"))); HdfsConfiguration hdfsConfiguration = new HdfsConfiguration(); properties.forEach(hdfsConfiguration::set); jdbcCatalog.setConf(hdfsConfiguration); diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergHiveCachedClientPool.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java similarity index 97% rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergHiveCachedClientPool.java rename to iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java index c6634c56355..629e9b97c5c 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergHiveCachedClientPool.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java @@ -1,4 +1,4 @@ -/* +package org.apache.gravitino.iceberg.common.utils;/* * 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 @@ -18,8 +18,6 @@ * */ -package org.apache.gravitino.catalog.lakehouse.iceberg; - import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.Scheduler; @@ -53,7 +51,7 @@ * Referred from Apache Iceberg's CachedClientPool implementation * hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java * - *

IcebergHiveCachedClientPool is used for every Iceberg catalog with Hive backend, I changed the + *

org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool is used for every Iceberg catalog with Hive backend, I changed the * method clientPool() from * *

{@code
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergConfig.java b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
similarity index 96%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergConfig.java
rename to iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
index 570158bed3f..31dc27931bc 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/TestIcebergConfig.java
+++ b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
@@ -17,10 +17,11 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg;
+package org.apache.gravitino.iceberg.common;
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.server.web.JettyServerConfig;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/TestIcebergCatalogUtil.java b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
similarity index 81%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/TestIcebergCatalogUtil.java
rename to iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
index 1f74e64f52b..65807ceec95 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/TestIcebergCatalogUtil.java
+++ b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
@@ -17,12 +17,12 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.utils;
+package org.apache.gravitino.iceberg.common.utils;
 
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.IcebergConstants;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.hive.HiveCatalog;
@@ -62,10 +62,10 @@ void testLoadCatalog() {
     Map properties = new HashMap<>();
     properties.put(CatalogProperties.URI, "jdbc://0.0.0.0:3306");
     properties.put(CatalogProperties.WAREHOUSE_LOCATION, "test");
-    properties.put(IcebergCatalogPropertiesMetadata.GRAVITINO_JDBC_DRIVER, "org.sqlite.JDBC");
-    properties.put(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_USER, "test");
-    properties.put(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_PASSWORD, "test");
-    properties.put(IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_INITIALIZE, "false");
+    properties.put(IcebergConstants.GRAVITINO_JDBC_DRIVER, "org.sqlite.JDBC");
+    properties.put(IcebergConstants.ICEBERG_JDBC_USER, "test");
+    properties.put(IcebergConstants.ICEBERG_JDBC_PASSWORD, "test");
+    properties.put(IcebergConstants.ICEBERG_JDBC_INITIALIZE, "false");
     catalog = IcebergCatalogUtil.loadCatalogBackend("jdbc", properties);
     Assertions.assertTrue(catalog instanceof JdbcCatalog);
 
diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts
new file mode 100644
index 00000000000..0c24fd0706e
--- /dev/null
+++ b/iceberg/iceberg-rest-server/build.gradle.kts
@@ -0,0 +1,171 @@
+/*
+ * 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.
+ */
+description = "iceberg-rest-service"
+
+plugins {
+  `maven-publish`
+  id("java")
+  id("idea")
+}
+
+val scalaVersion: String = project.properties["scalaVersion"] as? String ?: extra["defaultScalaVersion"].toString()
+val sparkVersion: String = libs.versions.spark34.get()
+val sparkMajorVersion: String = sparkVersion.substringBeforeLast(".")
+val icebergVersion: String = libs.versions.iceberg.get()
+val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get()
+
+dependencies {
+  implementation(project(":core"))
+  implementation(project(":common"))
+  implementation(project(":iceberg:iceberg-common"))
+  implementation(project(":server-common"))
+  implementation(libs.bundles.iceberg)
+  implementation(libs.bundles.jetty)
+  implementation(libs.bundles.jersey)
+  implementation(libs.bundles.log4j)
+  implementation(libs.guava)
+  implementation(libs.jackson.annotations)
+  implementation(libs.jackson.databind)
+  implementation(libs.jackson.datatype.jdk8)
+  implementation(libs.jackson.datatype.jsr310)
+  implementation(libs.metrics.jersey2)
+
+  annotationProcessor(libs.lombok)
+
+  compileOnly(libs.lombok)
+
+  testImplementation(project(":integration-test-common", "testArtifacts"))
+
+  testImplementation("org.scala-lang.modules:scala-collection-compat_$scalaVersion:$scalaCollectionCompatVersion")
+  testImplementation("org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_$scalaVersion:$icebergVersion")
+  testImplementation("org.apache.spark:spark-sql_$scalaVersion:$sparkVersion") {
+    exclude("org.apache.avro")
+    exclude("org.apache.hadoop")
+    exclude("org.apache.zookeeper")
+    exclude("io.dropwizard.metrics")
+    exclude("org.rocksdb")
+  }
+
+  testImplementation(libs.jersey.test.framework.core) {
+    exclude(group = "org.junit.jupiter")
+  }
+  testImplementation(libs.jersey.test.framework.provider.jetty) {
+    exclude(group = "org.junit.jupiter")
+  }
+  testImplementation(libs.junit.jupiter.api)
+  testImplementation(libs.junit.jupiter.params)
+  testImplementation(libs.mockito.core)
+  testImplementation(libs.sqlite.jdbc)
+  testImplementation(libs.slf4j.api)
+  testImplementation(libs.testcontainers)
+
+  testRuntimeOnly(libs.junit.jupiter.engine)
+}
+
+tasks {
+  val runtimeJars by registering(Copy::class) {
+    from(configurations.runtimeClasspath)
+    into("build/libs")
+  }
+
+  val copyCatalogLibs by registering(Copy::class) {
+    dependsOn("jar", "runtimeJars")
+    from("build/libs")
+    into("$rootDir/distribution/package/catalogs/lakehouse-iceberg/libs")
+  }
+
+  val copyCatalogConfig by registering(Copy::class) {
+    from("src/main/resources")
+    into("$rootDir/distribution/package/catalogs/lakehouse-iceberg/conf")
+
+    include("lakehouse-iceberg.conf")
+    include("core-site.xml.template")
+    include("hdfs-site.xml.template")
+
+    rename { original ->
+      if (original.endsWith(".template")) {
+        original.replace(".template", "")
+      } else {
+        original
+      }
+    }
+
+    exclude { details ->
+      details.file.isDirectory()
+    }
+  }
+
+  register("copyLibAndConfig", Copy::class) {
+    dependsOn(copyCatalogLibs, copyCatalogConfig)
+  }
+}
+
+tasks.test {
+  val skipUTs = project.hasProperty("skipTests")
+  if (skipUTs) {
+    // Only run integration tests
+    include("**/integration/**")
+  }
+
+  val skipITs = project.hasProperty("skipITs")
+  if (skipITs) {
+    // Exclude integration tests
+    exclude("**/integration/**")
+  } else {
+    dependsOn(tasks.jar)
+
+    doFirst {
+      environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.12")
+      environment("GRAVITINO_CI_KERBEROS_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-kerberos-hive:0.1.3")
+    }
+
+    val init = project.extra.get("initIntegrationTest") as (Test) -> Unit
+    init(this)
+  }
+}
+
+tasks.clean {
+  delete("spark-warehouse")
+}
+
+tasks.getByName("generateMetadataFileForMavenJavaPublication") {
+  dependsOn("copyDepends")
+}
+
+tasks {
+  val copyDepends by registering(Copy::class) {
+    from(configurations.runtimeClasspath)
+    into("build/libs")
+  }
+  jar {
+    finalizedBy(copyDepends)
+  }
+
+  register("copyLibs", Copy::class) {
+    dependsOn(copyDepends, "build")
+    from("build/libs")
+    into("$rootDir/distribution/package/iceberg-rest-server/libs")
+  }
+
+  register("copyLibsToStandalonePackage", Copy::class) {
+    dependsOn(copyDepends, "build")
+    from("build/libs")
+    into("$rootDir/distribution/iceberg-rest-server/libs")
+  }
+}
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergRESTService.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
similarity index 84%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergRESTService.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
index c722db8b250..722b0852bec 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergRESTService.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg;/*
  * 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
@@ -17,16 +17,15 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg;
-
 import java.util.Map;
 import javax.servlet.Servlet;
 import org.apache.gravitino.GravitinoEnv;
 import org.apache.gravitino.auxiliary.GravitinoAuxiliaryService;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergExceptionMapper;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergObjectMapperProvider;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics.IcebergMetricsManager;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergExceptionMapper;
+import org.apache.gravitino.iceberg.service.IcebergObjectMapperProvider;
+import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsManager;
 import org.apache.gravitino.metrics.MetricsSystem;
 import org.apache.gravitino.metrics.source.MetricsSource;
 import org.apache.gravitino.server.web.HttpServerMetricsSource;
@@ -39,9 +38,9 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class IcebergRESTService implements GravitinoAuxiliaryService {
+public class RESTService implements GravitinoAuxiliaryService {
 
-  private static Logger LOG = LoggerFactory.getLogger(IcebergRESTService.class);
+  private static Logger LOG = LoggerFactory.getLogger(RESTService.class);
 
   private JettyServer server;
 
@@ -58,7 +57,7 @@ private void initServer(IcebergConfig icebergConfig) {
     server.initialize(serverConfig, SERVICE_NAME, false /* shouldEnableUI */);
 
     ResourceConfig config = new ResourceConfig();
-    config.packages("org.apache.gravitino.catalog.lakehouse.iceberg.web.rest");
+    config.packages("org.apache.gravitino.iceberg.service.rest");
 
     config.register(IcebergObjectMapperProvider.class).register(JacksonFeature.class);
     config.register(IcebergExceptionMapper.class);
@@ -92,7 +91,7 @@ public String shortName() {
   public void serviceInit(Map properties) {
     IcebergConfig icebergConfig = new IcebergConfig(properties);
     initServer(icebergConfig);
-    LOG.info("Iceberg REST service inited");
+    LOG.info("Iceberg REST service init.");
   }
 
   @Override
@@ -121,4 +120,10 @@ public void serviceStop() throws Exception {
       icebergMetricsManager.close();
     }
   }
+
+  public void join() {
+    if (server != null) {
+      server.join();
+    }
+  }
 }
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergExceptionMapper.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
similarity index 98%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergExceptionMapper.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
index 5191e41e588..839baf5324e 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergExceptionMapper.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web;
-
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 import javax.ws.rs.core.Response;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapper.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
similarity index 96%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapper.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
index 5243c432ebd..12f35888eb5 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapper.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web;
-
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.PropertyAccessor;
 import com.fasterxml.jackson.databind.DeserializationFeature;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapperProvider.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
similarity index 94%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapperProvider.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
index c1687946108..b9d4b4b8fb9 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergObjectMapperProvider.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service;/*
  * 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
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import javax.ws.rs.ext.ContextResolver;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergRestUtils.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
similarity index 97%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergRestUtils.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
index 5387e9562b9..53f0ef8021d 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/IcebergRestUtils.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service;/*
  * 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
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web;
 
 import java.time.Instant;
 import java.time.LocalDateTime;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/DummyMetricsStore.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
similarity index 94%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/DummyMetricsStore.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
index f99532f53cc..833e18296cf 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/DummyMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
-
 import java.time.Instant;
 import java.util.Map;
 import org.apache.iceberg.metrics.MetricsReport;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsFormatter.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
similarity index 90%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsFormatter.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
index ae04983f882..6a0988f54b3 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsFormatter.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,11 +17,9 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
-
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergObjectMapper;
+import org.apache.gravitino.iceberg.service.IcebergObjectMapper;
 import org.apache.iceberg.metrics.MetricsReport;
 
 public class IcebergMetricsFormatter {
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsManager.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
similarity index 91%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsManager.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
index d5ae4b62bd3..81e55fff6df 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsManager.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -30,22 +28,19 @@
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergRestUtils;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergRestUtils;
 import org.apache.iceberg.metrics.MetricsReport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class IcebergMetricsManager {
   private static final Logger LOG = LoggerFactory.getLogger(IcebergTableOps.class);
-  public static final String ICEBERG_METRICS_STORE = "metricsStore";
-  public static final String ICEBERG_METRICS_STORE_RETAIN_DAYS = "metricsStoreRetainDays";
-  public static final String ICEBERG_METRICS_QUEUE_CAPACITY = "metricsQueueCapacity";
 
-  // Register IcebergMetricsStore's short name to its full qualified class name in the map. So
+  // Register org.apache.metrics.service.iceberg.gravitino.IcebergMetricsStore's short name to its full qualified class name in the map. So
   // that user doesn't need to specify the full qualified class name when creating an
-  // IcebergMetricsStore.
+  // org.apache.metrics.service.iceberg.gravitino.IcebergMetricsStore.
   private static final ImmutableMap ICEBERG_METRICS_STORE_NAMES =
       ImmutableMap.of(
           DummyMetricsStore.ICEBERG_METRICS_STORE_DUMMY_NAME,
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsStore.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
similarity index 96%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsStore.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
index cdc98329293..03fae0ee015 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/IcebergMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
-
 import java.io.IOException;
 import java.time.Instant;
 import java.util.Map;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergConfigOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
similarity index 92%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergConfigOperations.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
index c2217afcf1a..3ec38c29ea5 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergConfigOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -16,8 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
-
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
 import javax.servlet.http.HttpServletRequest;
@@ -28,7 +26,7 @@
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergRestUtils;
+import org.apache.gravitino.iceberg.service.IcebergRestUtils;
 import org.apache.gravitino.metrics.MetricNames;
 import org.apache.iceberg.rest.responses.ConfigResponse;
 
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergNamespaceOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
similarity index 95%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergNamespaceOperations.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
index 6f4e84683c5..183ea9fa8c8 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergNamespaceOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
@@ -34,8 +33,8 @@
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergRestUtils;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergRestUtils;
 import org.apache.gravitino.metrics.MetricNames;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.rest.RESTUtil;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
similarity index 95%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableOperations.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
index 74f6eca89f8..bc950697901 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
+
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import javax.inject.Inject;
@@ -37,10 +37,10 @@
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergObjectMapper;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergRestUtils;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics.IcebergMetricsManager;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergObjectMapper;
+import org.apache.gravitino.iceberg.service.IcebergRestUtils;
+import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsManager;
 import org.apache.gravitino.metrics.MetricNames;
 import org.apache.iceberg.catalog.TableIdentifier;
 import org.apache.iceberg.rest.RESTUtil;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableRenameOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
similarity index 90%
rename from catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableRenameOperations.java
rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
index 64a33b187ee..04cdbed017b 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTableRenameOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
@@ -29,8 +28,8 @@
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergRestUtils;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergRestUtils;
 import org.apache.gravitino.metrics.MetricNames;
 import org.apache.iceberg.rest.requests.RenameTableRequest;
 
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java
new file mode 100644
index 00000000000..7853ae6d41b
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java
@@ -0,0 +1,105 @@
+/*
+ * 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.gravitino.server;
+
+import org.apache.gravitino.Config;
+import org.apache.gravitino.GravitinoEnv;
+import org.apache.gravitino.iceberg.RESTService;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.server.authentication.ServerAuthenticator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IcebergRESTServer {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergRESTServer.class);
+
+  public static final String CONF_FILE = "iceberg-rest-server.conf";
+
+  private final Config serverConfig;
+
+  private RESTService icebergRESTService;
+  private GravitinoEnv gravitinoEnv;
+
+  public IcebergRESTServer(Config config) {
+    this.serverConfig = config;
+    this.gravitinoEnv = GravitinoEnv.getInstance();
+    this.icebergRESTService = new RESTService();
+  }
+
+  public void initialize() {
+    gravitinoEnv.initialize(serverConfig, false);
+    icebergRESTService.serviceInit(
+        serverConfig.getConfigsWithPrefix(IcebergConfig.ICEBERG_CONFIG_PREFIX));
+    ServerAuthenticator.getInstance().initialize(serverConfig);
+  }
+
+  public void start() {
+    icebergRESTService.serviceStart();
+  }
+
+  public void join() {
+    icebergRESTService.join();
+  }
+
+  public void stop() throws Exception {
+    icebergRESTService.serviceStop();
+    LOG.info("Iceberg REST service stopped");
+  }
+
+  public static void main(String[] args) {
+    LOG.info("Starting Iceberg REST Server");
+    String confPath = System.getenv("GRAVITINO_TEST") == null ? "" : args[0];
+    ServerConfig serverConfig = ServerConfig.loadConfig(confPath, CONF_FILE);
+    IcebergRESTServer icebergRESTServer = new IcebergRESTServer(serverConfig);
+    icebergRESTServer.initialize();
+
+    try {
+      icebergRESTServer.start();
+    } catch (Exception e) {
+      LOG.error("Error while running jettyServer", e);
+      System.exit(-1);
+    }
+    LOG.info("Done, Iceberg REST server started.");
+
+    Runtime.getRuntime()
+        .addShutdownHook(
+            new Thread(
+                () -> {
+                  try {
+                    // Register some clean-up tasks that need to be done before shutting down
+                    Thread.sleep(serverConfig.get(ServerConfig.SERVER_SHUTDOWN_TIMEOUT));
+                  } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    LOG.error("Interrupted exception:", e);
+                  } catch (Exception e) {
+                    LOG.error("Error while running clean-up tasks in shutdown hook", e);
+                  }
+                }));
+    icebergRESTServer.join();
+
+    LOG.info("Shutting down Iceberg REST Server ... ");
+    try {
+      icebergRESTServer.stop();
+      LOG.info("Iceberg REST Server has shut down.");
+    } catch (Exception e) {
+      LOG.error("Error while stopping Iceberg REST Server", e);
+    }
+  }
+}
diff --git a/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService b/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService
new file mode 100644
index 00000000000..768b2fbd807
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService
@@ -0,0 +1,19 @@
+#
+# 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.gravitino.iceberg.RESTService
diff --git a/iceberg/iceberg-rest-server/src/main/resources/core-site.xml.template b/iceberg/iceberg-rest-server/src/main/resources/core-site.xml.template
new file mode 100644
index 00000000000..efa7db5adb6
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/main/resources/core-site.xml.template
@@ -0,0 +1,21 @@
+
+
+
+
diff --git a/iceberg/iceberg-rest-server/src/main/resources/hdfs-site.xml.template b/iceberg/iceberg-rest-server/src/main/resources/hdfs-site.xml.template
new file mode 100644
index 00000000000..a47ec72adb9
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/main/resources/hdfs-site.xml.template
@@ -0,0 +1,20 @@
+
+
+
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTHiveCatalogIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTHiveCatalogIT.java
similarity index 63%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTHiveCatalogIT.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTHiveCatalogIT.java
index 7e3ba0efe00..837b4096185 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTHiveCatalogIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTHiveCatalogIT.java
@@ -16,14 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
+package org.apache.gravitino.iceberg.integration.test;
 
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.integration.test.container.ContainerSuite;
 import org.apache.gravitino.integration.test.container.HiveContainer;
 import org.apache.gravitino.integration.test.util.GravitinoITUtils;
@@ -31,9 +29,6 @@
 import org.junit.jupiter.api.TestInstance;
 import org.junit.jupiter.api.TestInstance.Lifecycle;
 
-// Hive&Jdbc catalog must be tested with gravitino-docker-test env,
-// so we should create a separate class instead using junit `parameterized test`
-// to auto-generate catalog type
 @Tag("gravitino-docker-test")
 @TestInstance(Lifecycle.PER_CLASS)
 public class IcebergRESTHiveCatalogIT extends IcebergRESTServiceIT {
@@ -50,34 +45,25 @@ void initEnv() {
 
   @Override
   Map getCatalogConfig() {
-    Map customConfigs = new HashMap<>();
-    customConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_BACKEND.getKey(),
+    Map configMap = new HashMap<>();
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_BACKEND.getKey(),
         IcebergCatalogBackend.HIVE.toString().toLowerCase());
 
-    customConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_URI.getKey(),
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_URI.getKey(),
         String.format(
             "thrift://%s:%d",
             containerSuite.getHiveContainer().getContainerIpAddress(),
             HiveContainer.HIVE_METASTORE_PORT));
 
-    customConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_WAREHOUSE.getKey(),
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_WAREHOUSE.getKey(),
         GravitinoITUtils.genRandomName(
             String.format(
                 "hdfs://%s:%d/user/hive/warehouse-hive",
                 containerSuite.getHiveContainer().getContainerIpAddress(),
                 HiveContainer.HDFS_DEFAULTFS_PORT)));
-    return customConfigs;
+    return configMap;
   }
 }
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
similarity index 57%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
index a3abd86621a..7a2ced56c87 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
@@ -16,15 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.integration.test;
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
-
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.IcebergConstants;
 import java.util.HashMap;
 import java.util.Map;
-import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService;
 import org.apache.gravitino.integration.test.container.ContainerSuite;
 import org.apache.gravitino.integration.test.container.HiveContainer;
 import org.apache.gravitino.integration.test.util.GravitinoITUtils;
@@ -50,52 +48,28 @@ public Map getCatalogConfig() {
     Map configMap = new HashMap<>();
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_BACKEND.getKey(),
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_BACKEND.getKey(),
         IcebergCatalogBackend.JDBC.toString().toLowerCase());
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.JDBC_DRIVER.getKey(),
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.JDBC_DRIVER.getKey(),
         "org.sqlite.JDBC");
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_URI.getKey(),
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_URI.getKey(),
         "jdbc:sqlite::memory:");
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.JDBC_USER.getKey(),
-        "iceberg");
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConstants.ICEBERG_JDBC_USER, "iceberg");
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.JDBC_PASSWORD.getKey(),
-        "iceberg");
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConstants.ICEBERG_JDBC_PASSWORD, "iceberg");
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.JDBC_INIT_TABLES.getKey(),
-        "true");
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.JDBC_INIT_TABLES.getKey(), "true");
 
     configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_WAREHOUSE.getKey(),
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_WAREHOUSE.getKey(),
         GravitinoITUtils.genRandomName(
             String.format(
                 "hdfs://%s:%d/user/hive/warehouse-jdbc-sqlite",
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTMemoryCatalogIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTMemoryCatalogIT.java
new file mode 100644
index 00000000000..d1f6225a10b
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTMemoryCatalogIT.java
@@ -0,0 +1,42 @@
+/*
+ * 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.gravitino.iceberg.integration.test;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+
+public class IcebergRESTMemoryCatalogIT extends IcebergRESTServiceIT {
+
+  @Override
+  void initEnv() {}
+
+  @Override
+  Map getCatalogConfig() {
+    Map configMap = new HashMap<>();
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_BACKEND.getKey(),
+        IcebergCatalogBackend.MEMORY.toString().toLowerCase());
+
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + IcebergConfig.CATALOG_WAREHOUSE.getKey(), "/tmp/");
+    return configMap;
+  }
+}
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceBaseIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
similarity index 79%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceBaseIT.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
index bd0a88d32d2..d28cda6ea18 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceBaseIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.integration.test;/*
  * 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
@@ -17,8 +17,7 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
-
+import org.apache.gravitino.iceberg.integration.test.util.IcebergRESTServerManager;
 import com.google.common.collect.ImmutableList;
 import com.google.errorprone.annotations.FormatMethod;
 import java.util.ArrayList;
@@ -29,14 +28,9 @@
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.gravitino.Config;
-import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService;
-import org.apache.gravitino.integration.test.util.AbstractIT;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.server.web.JettyServerConfig;
-import org.apache.gravitino.utils.MapUtils;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SparkSession;
 import org.junit.jupiter.api.AfterAll;
@@ -50,21 +44,21 @@
  */
 
 @SuppressWarnings("FormatStringAnnotation")
-public abstract class IcebergRESTServiceBaseIT extends AbstractIT {
+public abstract class IcebergRESTServiceBaseIT {
   public static final Logger LOG = LoggerFactory.getLogger(IcebergRESTServiceBaseIT.class);
   private SparkSession sparkSession;
   protected IcebergCatalogBackend catalogType = IcebergCatalogBackend.MEMORY;
+  private IcebergRESTServerManager icebergRESTServerManager;
 
   @BeforeAll
   void initIcebergTestEnv() throws Exception {
-    // Start Gravitino docker container
+    // Start Hive HDFS if necessary
     initEnv();
-    // Inject Iceberg REST service config to gravitino.conf
+    // Start Iceberg REST server
+    this.icebergRESTServerManager = IcebergRESTServerManager.create();
+    // Inject the catalog specific config to iceberg-rest-server.conf
     registerIcebergCatalogConfig();
-
-    ignoreIcebergRestService = false;
-    // Start Gravitino server
-    AbstractIT.startIntegrationTest();
+    icebergRESTServerManager.startIcebergRESTServer();
     // Start Spark session
     initSparkEnv();
     LOG.info("Gravitino and Spark env started,{}", catalogType);
@@ -73,19 +67,10 @@ void initIcebergTestEnv() throws Exception {
   @AfterAll
   void stopIcebergTestEnv() throws Exception {
     stopSparkEnv();
-    AbstractIT.stopIntegrationTest();
+    icebergRESTServerManager.stopIcebergRESTServer();
     LOG.info("Gravitino and Spark env stopped,{}", catalogType);
   }
 
-  // AbstractIT#startIntegrationTest() is static, so we couldn't inject catalog info
-  // if startIntegrationTest() is auto invoked by Junit. So here we override
-  // startIntegrationTest() to disable the auto invoke by junit.
-  @BeforeAll
-  public static void startIntegrationTest() {}
-
-  @AfterAll
-  public static void stopIntegrationTest() {}
-
   boolean catalogTypeNotMemory() {
     return !catalogType.equals(IcebergCatalogBackend.MEMORY);
   }
@@ -96,20 +81,19 @@ boolean catalogTypeNotMemory() {
 
   private void registerIcebergCatalogConfig() {
     Map icebergConfigs = getCatalogConfig();
-    AbstractIT.registerCustomConfigs(icebergConfigs);
+    icebergRESTServerManager.registerCustomConfigs(icebergConfigs);
     LOG.info("Iceberg REST service config registered, {}", StringUtils.join(icebergConfigs));
   }
 
-  private static IcebergConfig buildIcebergConfig(Config config) {
-    Map m =
-        config.getConfigsWithPrefix(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX);
-    m = MapUtils.getPrefixMap(m, IcebergRESTService.SERVICE_NAME + ".");
-    return new IcebergConfig(m);
+  private int getServerPort() {
+    JettyServerConfig jettyServerConfig =
+        JettyServerConfig.fromConfig(
+            icebergRESTServerManager.getServerConfig(), IcebergConfig.ICEBERG_CONFIG_PREFIX);
+    return jettyServerConfig.getHttpPort();
   }
 
   private void initSparkEnv() {
-    IcebergConfig icebergConfig = buildIcebergConfig(serverConfig);
-    int port = icebergConfig.get(JettyServerConfig.WEBSERVER_HTTP_PORT);
+    int port = getServerPort();
     LOG.info("Iceberg REST server port:{}", port);
     String IcebergRESTUri = String.format("http://127.0.0.1:%d/iceberg/", port);
     sparkSession =
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
similarity index 94%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceIT.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
index ef1746f04a4..b5145b042c0 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/IcebergRESTServiceIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.integration.test;/*
  * 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
@@ -17,19 +17,13 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
-
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergCatalogBackend;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
 import org.apache.iceberg.exceptions.BadRequestException;
 import org.apache.iceberg.exceptions.ServiceFailureException;
 import org.apache.iceberg.exceptions.ValidationException;
@@ -41,16 +35,15 @@
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInstance;
 import org.junit.jupiter.api.TestInstance.Lifecycle;
 import org.junit.jupiter.api.condition.EnabledIf;
 
-@Tag("gravitino-docker-test")
+// Don't add @Tag("gravitino-docker-test"), because IcebergRESTMemoryCatalogIT don't need it.
 @SuppressWarnings("FormatStringAnnotation")
 @TestInstance(Lifecycle.PER_CLASS)
-public class IcebergRESTServiceIT extends IcebergRESTServiceBaseIT {
+public abstract class IcebergRESTServiceIT extends IcebergRESTServiceBaseIT {
 
   private static final String ICEBERG_REST_NS_PREFIX = "iceberg_rest_";
 
@@ -67,28 +60,6 @@ void cleanup() {
     purgeAllIcebergTestNamespaces();
   }
 
-  @Override
-  void initEnv() {}
-
-  @Override
-  Map getCatalogConfig() {
-    Map configMap = new HashMap<>();
-    configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_BACKEND.getKey(),
-        IcebergCatalogBackend.MEMORY.toString().toLowerCase());
-
-    configMap.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + IcebergRESTService.SERVICE_NAME
-            + "."
-            + IcebergConfig.CATALOG_WAREHOUSE.getKey(),
-        "/tmp/");
-    return configMap;
-  }
-
   private void purgeTable(String namespace, String table) {
     sql(String.format("DROP TABLE %s.%s PURGE", namespace, table));
   }
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java
new file mode 100644
index 00000000000..6367cb41fa9
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java
@@ -0,0 +1,166 @@
+/*
+ * 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.gravitino.iceberg.integration.test.util;
+
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
+import org.apache.gravitino.Config;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.integration.test.util.ITUtils;
+import org.apache.gravitino.rest.RESTUtils;
+import org.apache.gravitino.server.IcebergRESTServer;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.gravitino.server.ServerConfig;
+import org.apache.gravitino.server.web.JettyServerConfig;
+import org.apache.hc.client5.http.classic.methods.HttpGet;
+import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
+import org.apache.hc.client5.http.impl.classic.HttpClients;
+import org.apache.hc.core5.http.ClassicHttpResponse;
+import org.apache.iceberg.rest.RESTUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class IcebergRESTServerManager {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(IcebergRESTServerManager.class);
+
+  protected Map customConfigs = new HashMap<>();
+  protected Config serverConfig;
+  protected String checkUri;
+
+  public abstract Path getConfigDir();
+
+  public abstract Optional> doStartIcebergRESTServer() throws Exception;
+
+  public abstract void doStopIcebergRESTServer();
+
+  public static IcebergRESTServerManager create() {
+    String testMode = System.getProperty(ITUtils.TEST_MODE);
+    if (ITUtils.EMBEDDED_TEST_MODE.equals(testMode)) {
+      return new IcebergRESTServerManagerForEmbedded();
+    } else {
+      return new IcebergRESTServerManagerForDeploy();
+    }
+  }
+
+  public void registerCustomConfigs(Map configs) {
+    customConfigs.putAll(configs);
+  }
+
+  public Config getServerConfig() {
+    return serverConfig;
+  }
+
+  public void startIcebergRESTServer() throws Exception {
+    initServerConfig();
+    Optional> future = doStartIcebergRESTServer();
+
+    long beginTime = System.currentTimeMillis();
+    boolean started = false;
+
+    while (System.currentTimeMillis() - beginTime < 1000 * 60) {
+      started = isHttpServerUp(checkUri);
+      if (started || (future.isPresent() && future.get().isDone())) {
+        break;
+      }
+      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+    }
+    if (!started) {
+      try {
+        if (future.isPresent()) {
+          future.get().get(1, TimeUnit.SECONDS);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("IcebergRESTServer start failed", e);
+      }
+      throw new RuntimeException("Can not start IcebergRESTServer");
+    }
+  }
+
+  public void stopIcebergRESTServer() {
+    doStopIcebergRESTServer();
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+
+    long beginTime = System.currentTimeMillis();
+    boolean started = true;
+    while (System.currentTimeMillis() - beginTime < 3 * 1000 * 60) {
+      sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+      started = isHttpServerUp(checkUri);
+      if (!started) {
+        break;
+      }
+    }
+    if (started) {
+      throw new RuntimeException("Can not stop IcebergRESTServer");
+    }
+  }
+
+  private void customizeConfigFile(String configTempFileName, String configFileName)
+      throws IOException {
+    Map configMap = new HashMap<>();
+    configMap.put(
+        IcebergConfig.ICEBERG_CONFIG_PREFIX + JettyServerConfig.WEBSERVER_HTTP_PORT.getKey(),
+        String.valueOf(RESTUtils.findAvailablePort(2000, 3000)));
+
+    configMap.putAll(customConfigs);
+
+    ITUtils.rewriteConfigFile(configTempFileName, configFileName, configMap);
+  }
+
+  private void initServerConfig() throws IOException {
+    Path configDir = getConfigDir();
+    String gravitinoRootDir = System.getenv("GRAVITINO_ROOT_DIR");
+
+    Path configFile = Paths.get(configDir.toString(), IcebergRESTServer.CONF_FILE);
+    customizeConfigFile(
+        Paths.get(gravitinoRootDir, "conf", IcebergRESTServer.CONF_FILE + ".template").toString(),
+        configFile.toString());
+    this.serverConfig = new ServerConfig();
+    Properties properties = serverConfig.loadPropertiesFromFile(configFile.toFile());
+    serverConfig.loadFromProperties(properties);
+
+    LOG.info("Server config:{}.", serverConfig.getAllConfig());
+
+    JettyServerConfig jettyServerConfig =
+        JettyServerConfig.fromConfig(serverConfig, IcebergConfig.ICEBERG_CONFIG_PREFIX);
+    String host = jettyServerConfig.getHost();
+    int port = jettyServerConfig.getHttpPort();
+    this.checkUri = String.format("http://%s:%d/metrics", host, port);
+    LOG.info("Check uri:{}.", checkUri);
+  }
+
+  private static boolean isHttpServerUp(String testUrl) {
+    try (CloseableHttpClient httpClient = HttpClients.createDefault()) {
+      HttpGet request = new HttpGet(testUrl);
+      ClassicHttpResponse response = httpClient.execute(request, a -> a);
+      return response.getCode() == 200;
+    } catch (Exception e) {
+      LOG.warn("Check server failed: url:{}, error message:{}", testUrl, e.getMessage());
+      return false;
+    }
+  }
+}
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
new file mode 100644
index 00000000000..1c60a361617
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
@@ -0,0 +1,68 @@
+/*
+ * 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.gravitino.iceberg.integration.test.util;
+
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Optional;
+import java.util.concurrent.Future;
+import org.apache.gravitino.integration.test.util.CommandExecutor;
+import org.apache.gravitino.integration.test.util.JdbcDriverDownloader;
+import org.apache.gravitino.integration.test.util.ProcessData;
+import org.apache.gravitino.integration.test.util.ProcessData.TypesOfData;
+import org.testcontainers.shaded.com.google.common.collect.ImmutableMap;
+
+public class IcebergRESTServerManagerForDeploy extends IcebergRESTServerManager {
+
+  private static final String SCRIPT_NAME = "iceberg-rest-server.sh";
+  private Path icebergRESTServerHome;
+  private static final String SQLITE_DRIVER_DOWNLOAD_URL =
+      "https://repo1.maven.org/maven2/org/xerial/sqlite-jdbc/3.42.0.0/sqlite-jdbc-3.42.0.0.jar";
+
+  public IcebergRESTServerManagerForDeploy() {
+    String gravitinoRootDir = System.getenv("GRAVITINO_ROOT_DIR");
+    this.icebergRESTServerHome =
+        Paths.get(gravitinoRootDir, "distribution", "package", "iceberg-rest-server");
+  }
+
+  @Override
+  public Path getConfigDir() {
+    return Paths.get(icebergRESTServerHome.toString(), "conf");
+  }
+
+  @Override
+  public Optional> doStartIcebergRESTServer() throws Exception {
+    JdbcDriverDownloader.downloadJdbcDriver(
+        SQLITE_DRIVER_DOWNLOAD_URL, Paths.get(icebergRESTServerHome.toString(), "libs").toString());
+
+    String cmd = String.format("%s/bin/%s start", icebergRESTServerHome.toString(), SCRIPT_NAME);
+    CommandExecutor.executeCommandLocalHost(
+        cmd,
+        false,
+        ProcessData.TypesOfData.OUTPUT,
+        ImmutableMap.of("GRAVITINO_HOME", icebergRESTServerHome.toString()));
+    return Optional.empty();
+  }
+
+  @Override
+  public void doStopIcebergRESTServer() {
+    String cmd = String.format("%s/bin/%s stop", icebergRESTServerHome.toString(), SCRIPT_NAME);
+    CommandExecutor.executeCommandLocalHost(cmd, false, TypesOfData.ERROR);
+  }
+}
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java
new file mode 100644
index 00000000000..0928e86adda
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.gravitino.iceberg.integration.test.util;
+
+import org.apache.gravitino.server.IcebergRESTServer;
+import java.io.File;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+public class IcebergRESTServerManagerForEmbedded extends IcebergRESTServerManager {
+
+  private File mockConfDir;
+  private final ExecutorService executor;
+
+  public IcebergRESTServerManagerForEmbedded() {
+    try {
+      this.mockConfDir = Files.createTempDirectory("MiniIcebergRESTServer").toFile();
+      LOG.info("config dir:{}", mockConfDir.getAbsolutePath());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    mockConfDir.mkdirs();
+    mockConfDir.deleteOnExit();
+    this.executor = Executors.newSingleThreadExecutor();
+  }
+
+  @Override
+  public Path getConfigDir() {
+    return mockConfDir.toPath();
+  }
+
+  @Override
+  public Optional> doStartIcebergRESTServer() {
+    Future future =
+        executor.submit(
+            () -> {
+              try {
+                IcebergRESTServer.main(
+                    new String[] {
+                      Paths.get(mockConfDir.getAbsolutePath(), IcebergRESTServer.CONF_FILE)
+                          .toString()
+                    });
+              } catch (Exception e) {
+                LOG.error("Exception in startup mini IcebergRESTServer ", e);
+                throw new RuntimeException(e);
+              }
+            });
+
+    return Optional.of(future);
+  }
+
+  @Override
+  public void doStopIcebergRESTServer() {
+    executor.shutdownNow();
+  }
+}
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/TestIcebergExceptionMapper.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
similarity index 97%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/TestIcebergExceptionMapper.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
index 1a3dc3f587c..b13441cfea0 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/TestIcebergExceptionMapper.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service;/*
  * 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
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.gravitino.catalog.lakehouse.iceberg.web;
 
 import javax.ws.rs.core.Response;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/MemoryMetricsStore.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
similarity index 93%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/MemoryMetricsStore.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
index baf3b53e237..8abb982be4d 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/MemoryMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,10 +17,9 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
-
 import java.time.Instant;
 import java.util.Map;
+import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsStore;
 import org.apache.iceberg.metrics.MetricsReport;
 
 /** Store Iceberg metrics in memory, used for test */
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/TestIcebergMetricsManager.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
similarity index 89%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/TestIcebergMetricsManager.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
index 6543a6b77c4..43b26816838 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/metrics/TestIcebergMetricsManager.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.metrics;/*
  * 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
@@ -17,15 +17,15 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics;
 
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.testcontainers.shaded.org.awaitility.Awaitility.await;
 
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.IcebergConstants;
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
 import org.apache.iceberg.metrics.ImmutableCommitMetricsResult;
 import org.apache.iceberg.metrics.ImmutableCommitReport;
 import org.apache.iceberg.metrics.MetricsReport;
@@ -73,8 +73,7 @@ void testIcebergMetricsManager() {
   @Test
   void testIcebergMetricsManagerWithNotExistsStoreType() {
     IcebergConfig icebergConfig =
-        new IcebergConfig(
-            ImmutableMap.of(IcebergMetricsManager.ICEBERG_METRICS_STORE, "not-exists"));
+        new IcebergConfig(ImmutableMap.of(IcebergConstants.ICEBERG_METRICS_STORE, "not-exists"));
 
     Assertions.assertThrowsExactly(
         RuntimeException.class, () -> new IcebergMetricsManager(icebergConfig));
@@ -84,10 +83,7 @@ void testIcebergMetricsManagerWithNotExistsStoreType() {
   void testIcebergMetricsManagerWithMemoryStore() throws InterruptedException {
     Map properties =
         ImmutableMap.of(
-            IcebergMetricsManager.ICEBERG_METRICS_STORE,
-            "org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics.MemoryMetricsStore",
-            "a",
-            "b");
+            IcebergConstants.ICEBERG_METRICS_STORE, MemoryMetricsStore.class.getName(), "a", "b");
     IcebergConfig icebergConfig = new IcebergConfig(properties);
 
     IcebergMetricsManager icebergMetricsManager = new IcebergMetricsManager(icebergConfig);
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergRestTestUtil.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
similarity index 86%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergRestTestUtil.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
index b734db81d06..208664d6e7d 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergRestTestUtil.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -17,15 +17,13 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
-
 import java.util.logging.Level;
 import java.util.logging.Logger;
-import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConfig;
-import org.apache.gravitino.catalog.lakehouse.iceberg.ops.IcebergTableOps;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergExceptionMapper;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergObjectMapperProvider;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics.IcebergMetricsManager;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.common.ops.IcebergTableOps;
+import org.apache.gravitino.iceberg.service.IcebergExceptionMapper;
+import org.apache.gravitino.iceberg.service.IcebergObjectMapperProvider;
+import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsManager;
 import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.jackson.JacksonFeature;
 import org.glassfish.jersey.logging.LoggingFeature;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTestBase.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
similarity index 90%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTestBase.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
index 19947ab7529..1aac961ed1f 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/IcebergTestBase.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -17,10 +17,7 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
-
 import com.google.common.base.Joiner;
-import java.io.IOException;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Optional;
@@ -29,23 +26,14 @@
 import javax.ws.rs.core.Application;
 import javax.ws.rs.core.MediaType;
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.gravitino.catalog.lakehouse.iceberg.web.IcebergObjectMapperProvider;
-import org.apache.gravitino.rest.RESTUtils;
+import org.apache.gravitino.iceberg.service.IcebergObjectMapperProvider;
 import org.glassfish.jersey.server.ResourceConfig;
 import org.glassfish.jersey.test.JerseyTest;
-import org.glassfish.jersey.test.TestProperties;
 import org.junit.jupiter.api.Assertions;
 
 public class IcebergTestBase extends JerseyTest {
   @Override
   protected Application configure() {
-    try {
-      forceSet(
-          TestProperties.CONTAINER_PORT, String.valueOf(RESTUtils.findAvailablePort(2000, 3000)));
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-
     return new ResourceConfig();
   }
 
@@ -126,6 +114,7 @@ public Invocation.Builder getIcebergClientBuilder(
       }
     }
 
+
     return target
         .register(IcebergObjectMapperProvider.class)
         .request(MediaType.APPLICATION_JSON_TYPE)
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergConfig.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
similarity index 97%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergConfig.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
index dac356af5d0..e6a6d52f863 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergConfig.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
-
 import java.util.Optional;
 import javax.ws.rs.core.Application;
 import javax.ws.rs.core.MediaType;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergNamespaceOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
similarity index 99%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergNamespaceOperations.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
index 54fc6f97c7b..922c5421d44 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergNamespaceOperations.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -17,7 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Arrays;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergTableOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
similarity index 99%
rename from catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergTableOperations.java
rename to iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
index bbcb60ba57e..bff250cf033 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/rest/TestIcebergTableOperations.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
@@ -1,4 +1,4 @@
-/*
+package org.apache.gravitino.iceberg.service.rest;/*
  * 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
@@ -17,8 +17,6 @@
  * under the License.
  */
 
-package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest;
-
 import com.google.common.collect.ImmutableSet;
 import java.util.Optional;
 import java.util.Set;
diff --git a/iceberg/iceberg-rest-server/src/test/resources/log4j2.properties b/iceberg/iceberg-rest-server/src/test/resources/log4j2.properties
new file mode 100644
index 00000000000..8555c8487f5
--- /dev/null
+++ b/iceberg/iceberg-rest-server/src/test/resources/log4j2.properties
@@ -0,0 +1,73 @@
+#
+# 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.
+#
+
+# Set to debug or trace if log4j initialization is failing
+status = info
+
+# Name of the configuration
+name = ConsoleLogConfig
+
+# Console appender configuration
+appender.console.type = Console
+appender.console.name = consoleLogger
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p [%t] %c{1}:%L - %m%n
+
+# Log files location
+property.logPath = ${sys:gravitino.log.path:-build/iceberg-rest-server-integration-test.log}
+
+# File appender configuration
+appender.file.type = File
+appender.file.name = fileLogger
+appender.file.fileName = ${logPath}
+appender.file.layout.type = PatternLayout
+appender.file.layout.pattern = %d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n
+
+# Root logger level
+rootLogger.level = info
+
+# Root logger referring to console and file appenders
+rootLogger.appenderRef.stdout.ref = consoleLogger
+rootLogger.appenderRef.file.ref = fileLogger
+
+# File appender configuration for testcontainers
+appender.testcontainersFile.type = File
+appender.testcontainersFile.name = testcontainersLogger
+appender.testcontainersFile.fileName = build/testcontainers.log
+appender.testcontainersFile.layout.type = PatternLayout
+appender.testcontainersFile.layout.pattern = %d{yyyy-MM-dd HH:mm:ss.SSS} [%t] %-5p %c - %m%n
+
+# Logger for testcontainers
+logger.testcontainers.name = org.testcontainers
+logger.testcontainers.level = debug
+logger.testcontainers.additivity = false
+logger.testcontainers.appenderRef.file.ref = testcontainersLogger
+
+logger.tc.name = tc
+logger.tc.level = debug
+logger.tc.additivity = false
+logger.tc.appenderRef.file.ref = testcontainersLogger
+
+logger.docker.name = com.github.dockerjava
+logger.docker.level = warn
+logger.docker.additivity = false
+logger.docker.appenderRef.file.ref = testcontainersLogger
+
+logger.http.name = com.github.dockerjava.zerodep.shaded.org.apache.hc.client5.http.wire
+logger.http.level = off
diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
index 5f86d9a7221..98a7ea7e023 100644
--- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
+++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
@@ -57,6 +57,7 @@
  * server in the same JVM process.
  */
 public class MiniGravitino {
+
   private static final Logger LOG = LoggerFactory.getLogger(MiniGravitino.class);
   private MiniGravitinoContext context;
   private RESTClient restClient;
@@ -215,22 +216,15 @@ public Config getServerConfig() {
   Map getIcebergRestServiceConfigs() throws IOException {
     Map customConfigs = new HashMap<>();
 
-    String icebergJarPath =
-        Paths.get("catalogs", "catalog-lakehouse-iceberg", "build", "libs").toString();
+    String icebergJarPath = Paths.get("iceberg", "iceberg-rest-server", "build", "libs").toString();
     String icebergConfigPath =
-        Paths.get("catalogs", "catalog-lakehouse-iceberg", "src", "main", "resources").toString();
+        Paths.get("iceberg", "iceberg-rest-server", "src", "main", "resources").toString();
     customConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + "iceberg-rest"
-            + "."
-            + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH,
+        "gravitino.iceberg-rest." + AuxiliaryServiceManager.AUX_SERVICE_CLASSPATH,
         String.join(",", icebergJarPath, icebergConfigPath));
 
     customConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
-            + "iceberg-rest"
-            + "."
-            + JettyServerConfig.WEBSERVER_HTTP_PORT.getKey(),
+        "gravitino.iceberg-rest." + JettyServerConfig.WEBSERVER_HTTP_PORT.getKey(),
         String.valueOf(RESTUtils.findAvailablePort(3000, 4000)));
     return customConfigs;
   }
diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
index dc2aa4b80ff..7fcf5921cf2 100644
--- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
+++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
@@ -332,7 +332,7 @@ public static boolean isHttpServerUp(String testUrl) {
       ClassicHttpResponse response = httpClient.execute(request, a -> a);
       return response.getCode() == 200;
     } catch (Exception e) {
-      LOG.warn("Check Gravitino server failed: ", e);
+      LOG.warn("Check gravitino server failed: url:{}, error message:{}", testUrl, e.getMessage());
       return false;
     }
   }
diff --git a/server-common/build.gradle.kts b/server-common/build.gradle.kts
index 662ac4a5ee8..22d2eb29d72 100644
--- a/server-common/build.gradle.kts
+++ b/server-common/build.gradle.kts
@@ -51,3 +51,10 @@ dependencies {
   }
   testRuntimeOnly(libs.junit.jupiter.engine)
 }
+
+tasks {
+  test {
+    environment("GRAVITINO_HOME", rootDir.path)
+    environment("GRAVITINO_TEST", "true")
+  }
+}
diff --git a/server/src/main/java/org/apache/gravitino/server/ServerConfig.java b/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java
similarity index 70%
rename from server/src/main/java/org/apache/gravitino/server/ServerConfig.java
rename to server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java
index 072ee94ec0b..747aac00b18 100644
--- a/server/src/main/java/org/apache/gravitino/server/ServerConfig.java
+++ b/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java
@@ -22,6 +22,8 @@
 import org.apache.gravitino.config.ConfigBuilder;
 import org.apache.gravitino.config.ConfigConstants;
 import org.apache.gravitino.config.ConfigEntry;
+import java.io.File;
+import java.util.Properties;
 
 public class ServerConfig extends Config {
 
@@ -40,4 +42,20 @@ public ServerConfig(boolean loadDefaults) {
   public ServerConfig() {
     this(true);
   }
+
+  public static ServerConfig loadConfig(String confPath, String defaultConfigName) {
+    ServerConfig serverConfig = new ServerConfig();
+    try {
+      if (confPath.isEmpty()) {
+        // Load default conf
+        serverConfig.loadFromFile(defaultConfigName);
+      } else {
+        Properties properties = serverConfig.loadPropertiesFromFile(new File(confPath));
+        serverConfig.loadFromProperties(properties);
+      }
+    } catch (Exception exception) {
+      throw new IllegalArgumentException("Failed to load conf from file " + confPath, exception);
+    }
+    return serverConfig;
+  }
 }
diff --git a/server-common/src/main/java/org/apache/gravitino/server/web/JettyServerConfig.java b/server-common/src/main/java/org/apache/gravitino/server/web/JettyServerConfig.java
index ded01a8beb2..ead9645e8db 100644
--- a/server-common/src/main/java/org/apache/gravitino/server/web/JettyServerConfig.java
+++ b/server-common/src/main/java/org/apache/gravitino/server/web/JettyServerConfig.java
@@ -39,6 +39,7 @@
 
 public final class JettyServerConfig {
   private static final Logger LOG = LoggerFactory.getLogger(JettyServerConfig.class);
+  public static final String GRAVITINO_SERVER_CONFIG_PREFIX = "gravitino.server.webserver.";
   private static final String SPLITTER = ",";
   public static final int DEFAULT_ICEBERG_REST_SERVICE_HTTP_PORT = 9001;
   public static final int DEFAULT_ICEBERG_REST_SERVICE_HTTPS_PORT = 9433;
diff --git a/server/src/test/java/org/apache/gravitino/server/TestServerConfig.java b/server-common/src/test/java/org/apache/gravitino/server/TestServerConfig.java
similarity index 95%
rename from server/src/test/java/org/apache/gravitino/server/TestServerConfig.java
rename to server-common/src/test/java/org/apache/gravitino/server/TestServerConfig.java
index 43f6d6c366a..fc9193e0126 100644
--- a/server/src/test/java/org/apache/gravitino/server/TestServerConfig.java
+++ b/server-common/src/test/java/org/apache/gravitino/server/TestServerConfig.java
@@ -35,6 +35,7 @@
 import org.junit.jupiter.api.Test;
 
 public class TestServerConfig {
+
   @Test
   public void checkGravitinoConfFile()
       throws NoSuchFieldException, IllegalAccessException, IOException {
@@ -46,7 +47,7 @@ public void checkGravitinoConfFile()
         getConfigEntryFromClass(JettyServerConfig.class).entrySet().stream()
             .collect(
                 Collectors.toMap(
-                    kv -> GravitinoServer.WEBSERVER_CONF_PREFIX + kv.getKey(),
+                    kv -> JettyServerConfig.GRAVITINO_SERVER_CONFIG_PREFIX + kv.getKey(),
                     Map.Entry::getValue));
     configKeyMap.putAll(jettyConfigMap);
 
@@ -65,7 +66,8 @@ public void checkGravitinoConfFile()
     // `Configs`
     for (Map.Entry entry : properties.entrySet()) {
       String propKey = (String) entry.getKey();
-      if (propKey.startsWith(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX)) {
+      if (propKey.startsWith(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX)
+          || propKey.startsWith("gravitino.iceberg-rest.")) {
         continue;
       }
       Assertions.assertTrue(
diff --git a/server/src/main/java/org/apache/gravitino/server/GravitinoServer.java b/server/src/main/java/org/apache/gravitino/server/GravitinoServer.java
index cf1bb7aac8c..43e36bab914 100644
--- a/server/src/main/java/org/apache/gravitino/server/GravitinoServer.java
+++ b/server/src/main/java/org/apache/gravitino/server/GravitinoServer.java
@@ -77,7 +77,7 @@ public GravitinoServer(ServerConfig config) {
   }
 
   public void initialize() {
-    gravitinoEnv.initialize(serverConfig);
+    gravitinoEnv.initialize(serverConfig, true);
 
     JettyServerConfig jettyServerConfig =
         JettyServerConfig.fromConfig(serverConfig, WEBSERVER_CONF_PREFIX);
diff --git a/settings.gradle.kts b/settings.gradle.kts
index 5cc787807d0..07af7ac2ac9 100644
--- a/settings.gradle.kts
+++ b/settings.gradle.kts
@@ -45,6 +45,8 @@ include(
   "clients:filesystem-hadoop3-runtime",
   "clients:client-python"
 )
+include("iceberg:iceberg-common")
+include("iceberg:iceberg-rest-server")
 include("trino-connector")
 include("spark-connector:spark-common")
 // kyuubi hive connector doesn't support 2.13 for Spark3.3
diff --git a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/SparkEnvIT.java b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/SparkEnvIT.java
index 5eb4a203e50..d9cdd8435d2 100644
--- a/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/SparkEnvIT.java
+++ b/spark-connector/spark-common/src/test/java/org/apache/gravitino/spark/connector/integration/test/SparkEnvIT.java
@@ -26,7 +26,6 @@
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.gravitino.Catalog;
-import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
 import org.apache.gravitino.client.GravitinoMetalake;
 import org.apache.gravitino.integration.test.container.ContainerSuite;
 import org.apache.gravitino.integration.test.container.HiveContainer;
@@ -48,6 +47,7 @@
 
 /** Setup Hive, Gravitino, Spark, Metalake environment to execute SparkSQL. */
 public abstract class SparkEnvIT extends SparkUtilIT {
+
   private static final Logger LOG = LoggerFactory.getLogger(SparkEnvIT.class);
   private static final ContainerSuite containerSuite = ContainerSuite.getInstance();
 
@@ -155,19 +155,19 @@ private void initIcebergRestServiceEnv() {
     ignoreIcebergRestService = false;
     Map icebergRestServiceConfigs = new HashMap<>();
     icebergRestServiceConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
+        "gravitino."
             + icebergRestServiceName
             + "."
             + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND,
         IcebergPropertiesConstants.ICEBERG_CATALOG_BACKEND_HIVE);
     icebergRestServiceConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
+        "gravitino."
             + icebergRestServiceName
             + "."
             + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI,
         hiveMetastoreUri);
     icebergRestServiceConfigs.put(
-        AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX
+        "gravitino."
             + icebergRestServiceName
             + "."
             + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE,
@@ -213,8 +213,7 @@ private void initSparkEnv() {
   private String getIcebergRestServiceUri() {
     JettyServerConfig jettyServerConfig =
         JettyServerConfig.fromConfig(
-            serverConfig,
-            AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + icebergRestServiceName + ".");
+            serverConfig, String.format("gravitino.%s.", icebergRestServiceName));
     return String.format(
         "http://%s:%d/iceberg/", jettyServerConfig.getHost(), jettyServerConfig.getHttpPort());
   }
diff --git a/spark-connector/v3.3/spark/build.gradle.kts b/spark-connector/v3.3/spark/build.gradle.kts
index 971c9c42244..daaf0ae2d58 100644
--- a/spark-connector/v3.3/spark/build.gradle.kts
+++ b/spark-connector/v3.3/spark/build.gradle.kts
@@ -151,6 +151,7 @@ tasks.test {
     dependsOn(tasks.jar)
     dependsOn(":catalogs:catalog-lakehouse-iceberg:jar")
     dependsOn(":catalogs:catalog-hive:jar")
+    dependsOn(":iceberg:iceberg-rest-server:jar")
 
     doFirst {
       environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.13")
diff --git a/spark-connector/v3.4/spark/build.gradle.kts b/spark-connector/v3.4/spark/build.gradle.kts
index 59539474b49..26beeb18cf5 100644
--- a/spark-connector/v3.4/spark/build.gradle.kts
+++ b/spark-connector/v3.4/spark/build.gradle.kts
@@ -151,6 +151,7 @@ tasks.test {
     dependsOn(tasks.jar)
     dependsOn(":catalogs:catalog-lakehouse-iceberg:jar")
     dependsOn(":catalogs:catalog-hive:jar")
+    dependsOn(":iceberg:iceberg-rest-server:jar")
 
     doFirst {
       environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.13")
diff --git a/spark-connector/v3.5/spark/build.gradle.kts b/spark-connector/v3.5/spark/build.gradle.kts
index 1e285621d26..33c87e23d9d 100644
--- a/spark-connector/v3.5/spark/build.gradle.kts
+++ b/spark-connector/v3.5/spark/build.gradle.kts
@@ -153,6 +153,7 @@ tasks.test {
     dependsOn(tasks.jar)
     dependsOn(":catalogs:catalog-lakehouse-iceberg:jar")
     dependsOn(":catalogs:catalog-hive:jar")
+    dependsOn(":iceberg:iceberg-rest-server:jar")
 
     doFirst {
       environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.13")

From 5a19394a07d1e9cc2f2fdee07644abc0345cf3ee Mon Sep 17 00:00:00 2001
From: fanng 
Date: Tue, 16 Jul 2024 16:03:11 +0800
Subject: [PATCH 02/11] fix comment

---
 LICENSE                                       |  4 +-
 bin/iceberg-rest-server.sh                    |  2 +-
 build.gradle.kts                              |  4 +-
 .../lakehouse/iceberg/IcebergConstants.java   |  6 +++
 .../build.gradle.kts                          |  6 ++-
 .../IcebergCatalogPropertiesMetadata.java     |  3 +-
 ...vitino.auxiliary.GravitinoAuxiliaryService | 19 -------
 .../test/CatalogIcebergKerberosHiveIT.java    |  2 -
 .../test/CatalogIcebergRestIT.java            |  4 +-
 conf/gravitino.conf.template                  |  2 +-
 .../TestAuxiliaryServiceManager.java          |  2 +-
 iceberg/iceberg-common/build.gradle.kts       |  1 +
 .../iceberg/common/IcebergConfig.java         |  1 +
 .../iceberg/common/IcebergConstants.java      | 41 ---------------
 .../kerberos/HiveBackendProxy.java            |  3 +-
 .../kerberos/KerberosConfig.java              |  1 +
 .../iceberg/common/ops/IcebergTableOps.java   |  4 +-
 .../common/utils/IcebergCatalogUtil.java      |  8 ++-
 .../utils/IcebergHiveCachedClientPool.java    |  5 +-
 .../iceberg/common/TestIcebergConfig.java     |  1 -
 .../common/utils/TestIcebergCatalogUtil.java  |  4 +-
 iceberg/iceberg-rest-server/build.gradle.kts  |  1 +
 .../apache/gravitino/iceberg/RESTService.java |  3 +-
 .../service/IcebergExceptionMapper.java       |  3 +-
 .../iceberg/service/IcebergObjectMapper.java  |  3 +-
 .../service/IcebergObjectMapperProvider.java  |  3 +-
 .../iceberg/service/IcebergRestUtils.java     |  3 +-
 .../service/metrics/DummyMetricsStore.java    |  3 +-
 .../metrics/IcebergMetricsFormatter.java      |  3 +-
 .../metrics/IcebergMetricsManager.java        |  5 +-
 .../service/metrics/IcebergMetricsStore.java  |  3 +-
 .../service/rest/IcebergConfigOperations.java |  4 +-
 .../rest/IcebergNamespaceOperations.java      |  3 +-
 .../service/rest/IcebergTableOperations.java  |  4 +-
 .../rest/IcebergTableRenameOperations.java    |  3 +-
 ...itino.auxiliary.GravitinoAuxiliaryService} |  0
 .../test/IcebergRESTJdbcCatalogIT.java        |  6 +--
 .../test/IcebergRESTServiceBaseIT.java        |  5 +-
 .../test/IcebergRESTServiceIT.java            |  3 +-
 .../test/util/IcebergRESTServerManager.java   | 37 ++++----------
 .../IcebergRESTServerManagerForDeploy.java    |  6 +--
 .../IcebergRESTServerManagerForEmbedded.java  |  2 +-
 .../service/TestIcebergExceptionMapper.java   |  3 +-
 .../service/metrics/MemoryMetricsStore.java   |  5 +-
 .../metrics/TestIcebergMetricsManager.java    |  8 +--
 .../service/rest/IcebergRestTestUtil.java     |  4 +-
 .../iceberg/service/rest/IcebergTestBase.java |  4 +-
 .../service/rest/TestIcebergConfig.java       |  3 +-
 .../rest/TestIcebergNamespaceOperations.java  |  4 +-
 .../rest/TestIcebergTableOperations.java      |  4 +-
 .../integration/test/MiniGravitino.java       |  6 +--
 .../integration/test/util/AbstractIT.java     | 26 +---------
 .../integration/test/util/HttpUtils.java      | 50 +++++++++++++++++++
 .../apache/gravitino/server/ServerConfig.java |  4 +-
 spark-connector/v3.3/spark/build.gradle.kts   |  1 +
 55 files changed, 168 insertions(+), 180 deletions(-)
 delete mode 100644 catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService
 delete mode 100644 iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java
 rename iceberg/iceberg-rest-server/src/main/resources/META-INF/services/{com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService => org.apache.gravitino.auxiliary.GravitinoAuxiliaryService} (100%)
 create mode 100644 integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/HttpUtils.java

diff --git a/LICENSE b/LICENSE
index 169b5ef9964..2d4b82eb2be 100644
--- a/LICENSE
+++ b/LICENSE
@@ -232,10 +232,8 @@
    ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/miniHMS/MiniHiveMetastoreService.java
    ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/miniHMS/ScriptRunner.java
    ./catalogs/catalog-hive/src/test/java/org/apache/gravitino/catalog/hive/TestCachedClientPool.java
-   ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/web/org.apache.service.iceberg.gravitino.IcebergExceptionMapper.java
    ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/converter/DescribeIcebergSortOrderVisitor.java
    ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/utils/IcebergTablePropertiesUtil.java
-   ./catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool.java
    ./clients/client-java/src/main/java/org/apache/gravitino/client/HTTPClient.java
    ./clients/client-java/src/main/java/org/apache/gravitino/client/RESTClient.java
    ./clients/client-java/src/test/java/org/apache/gravitino/client/TestHTTPClient.java
@@ -245,6 +243,8 @@
    ./core/src/main/java/org/apache/gravitino/utils/ClientPoolImpl.java
    ./common/src/main/java/org/apache/gravitino/dto/responses/OAuth2TokenResponse.java
    ./clients/client-java/src/main/java/org/apache/gravitino/client/OAuth2ClientUtil.java
+   ./iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
+   ./iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java
    ./gradlew
 
    Apache Paimon
diff --git a/bin/iceberg-rest-server.sh b/bin/iceberg-rest-server.sh
index c4b192d1d38..15c7dec5bb9 100755
--- a/bin/iceberg-rest-server.sh
+++ b/bin/iceberg-rest-server.sh
@@ -143,7 +143,7 @@ function stop() {
 
 HOSTNAME=$(hostname)
 GRAVITINO_OUTFILE="${GRAVITINO_LOG_DIR}/iceberg-rest-server.out"
-GRAVITINO_SERVER_NAME=com.datastrato.gravitino.server.IcebergRESTServer
+GRAVITINO_SERVER_NAME=org.apache.gravitino.server.IcebergRESTServer
 GRAVITINO_SIMPLE_SERVER_NAME=iceberg-rest-server
 
 JAVA_OPTS+=" -Dfile.encoding=UTF-8"
diff --git a/build.gradle.kts b/build.gradle.kts
index 3357881bbe2..da81b276a94 100644
--- a/build.gradle.kts
+++ b/build.gradle.kts
@@ -686,7 +686,7 @@ tasks {
   register("copySubprojectDependencies", Copy::class) {
     subprojects.forEach() {
       if (!it.name.startsWith("catalog") &&
-        !it.name.startsWith("client") && !it.name.startsWith("filesystem") && !it.name.startsWith("spark") && !it.name.startsWith("iceberg-") && it.name != "trino-connector" &&
+        !it.name.startsWith("client") && !it.name.startsWith("filesystem") && !it.name.startsWith("spark") && !it.name.startsWith("iceberg") && it.name != "trino-connector" &&
         it.name != "integration-test" && it.name != "bundled-catalog" && it.name != "flink-connector"
       ) {
         from(it.configurations.runtimeClasspath)
@@ -701,7 +701,7 @@ tasks {
         !it.name.startsWith("client") &&
         !it.name.startsWith("filesystem") &&
         !it.name.startsWith("spark") &&
-        !it.name.startsWith("iceberg-") &&
+        !it.name.startsWith("iceberg") &&
         it.name != "trino-connector" &&
         it.name != "integration-test" &&
         it.name != "bundled-catalog" &&
diff --git a/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConstants.java b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConstants.java
index abb3a39ebbb..07417fd3c1b 100644
--- a/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConstants.java
+++ b/catalogs/catalog-common/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergConstants.java
@@ -46,4 +46,10 @@ public class IcebergConstants {
   public static final String PROVIDER = "provider";
   public static final String FORMAT = "format";
   public static final String FORMAT_VERSION = "format-version";
+
+  public static final String ICEBERG_METRICS_STORE = "metricsStore";
+  public static final String ICEBERG_METRICS_STORE_RETAIN_DAYS = "metricsStoreRetainDays";
+  public static final String ICEBERG_METRICS_QUEUE_CAPACITY = "metricsQueueCapacity";
+
+  public static final String GRAVITINO_ICEBERG_REST_SERVICE_NAME = "iceberg-rest";
 }
diff --git a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts
index f046be1f9d9..997f70b855a 100644
--- a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts
+++ b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts
@@ -70,8 +70,6 @@ dependencies {
     exclude("org.rocksdb")
   }
 
-  testImplementation(libs.bundles.log4j)
-  testImplementation(libs.bundles.jersey)
   testImplementation(libs.hadoop2.common) {
     exclude("com.github.spotbugs")
   }
@@ -102,6 +100,10 @@ tasks {
     into("build/libs")
   }
 
+  jar {
+    finalizedBy("runtimeJars")
+  }
+
   val copyCatalogLibs by registering(Copy::class) {
     dependsOn("jar", "runtimeJars")
     from("build/libs")
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java
index abdc29bf438..8f887816d55 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java
+++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/org/apache/gravitino/catalog/lakehouse/iceberg/IcebergCatalogPropertiesMetadata.java
@@ -21,8 +21,6 @@
 import static org.apache.gravitino.connector.PropertyEntry.enumImmutablePropertyEntry;
 import static org.apache.gravitino.connector.PropertyEntry.stringRequiredPropertyEntry;
 
-import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
-import org.apache.gravitino.iceberg.common.IcebergConstants;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -31,6 +29,7 @@
 import java.util.Map;
 import org.apache.gravitino.connector.BaseCatalogPropertiesMetadata;
 import org.apache.gravitino.connector.PropertyEntry;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
 import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig;
 import org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig;
 
diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService b/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService
deleted file mode 100644
index 543d06097fc..00000000000
--- a/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService
+++ /dev/null
@@ -1,19 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#  http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-#
-org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java
index cd220546ae6..e20cc97f747 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java
+++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergKerberosHiveIT.java
@@ -19,8 +19,6 @@
 package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
 
 import static org.apache.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX;
-
-
 import static org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig.AUTH_TYPE_KEY;
 import static org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig.IMPERSONATION_ENABLE_KEY;
 import static org.apache.gravitino.iceberg.common.authentication.kerberos.KerberosConfig.KET_TAB_URI_KEY;
diff --git a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java
index a7f6efd6a07..808da5cfc1f 100644
--- a/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java
+++ b/catalogs/catalog-lakehouse-iceberg/src/test/java/org/apache/gravitino/catalog/lakehouse/iceberg/integration/test/CatalogIcebergRestIT.java
@@ -18,9 +18,9 @@
  */
 package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test;
 
-import org.apache.gravitino.iceberg.common.IcebergConfig;
-import org.apache.gravitino.iceberg.common.IcebergConstants;
 import java.util.Map;
+import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.integration.test.container.HiveContainer;
 import org.apache.gravitino.server.web.JettyServerConfig;
 import org.junit.jupiter.api.Tag;
diff --git a/conf/gravitino.conf.template b/conf/gravitino.conf.template
index dab4d86cc92..448f7a5973b 100644
--- a/conf/gravitino.conf.template
+++ b/conf/gravitino.conf.template
@@ -63,7 +63,7 @@ gravitino.catalog.cache.evictionIntervalMs = 3600000
 # Auxiliary service names, separate by ','
 gravitino.auxService.names = iceberg-rest
 # Iceberg REST service classpath
-gravitino.iceberg-rest.classpath = iceberg-rest-server/libs
+gravitino.iceberg-rest.classpath = iceberg-rest-server/libs, conf
 # Iceberg REST service host
 gravitino.iceberg-rest.host = 0.0.0.0
 # Iceberg REST service http port
diff --git a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java
index 9582d1b8de5..578e2f84454 100644
--- a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java
+++ b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java
@@ -29,9 +29,9 @@
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Collections;
+import java.util.Map;
 import org.apache.gravitino.Config;
 import org.apache.gravitino.utils.IsolatedClassLoader;
-import java.util.Map;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
diff --git a/iceberg/iceberg-common/build.gradle.kts b/iceberg/iceberg-common/build.gradle.kts
index 415d858b88a..10dd4db0ab0 100644
--- a/iceberg/iceberg-common/build.gradle.kts
+++ b/iceberg/iceberg-common/build.gradle.kts
@@ -25,6 +25,7 @@ plugins {
 }
 
 dependencies {
+  implementation(project(":catalogs:bundled-catalog", configuration = "shadow"))
   implementation(project(":core"))
   implementation(project(":common"))
   implementation(project(":server-common"))
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java
index 657c0cc9a46..bad69675494 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConfig.java
@@ -24,6 +24,7 @@
 import java.util.Optional;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.gravitino.Config;
+import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
 import org.apache.gravitino.config.ConfigBuilder;
 import org.apache.gravitino.config.ConfigConstants;
 import org.apache.gravitino.config.ConfigEntry;
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java
deleted file mode 100644
index b3a3e738be8..00000000000
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/IcebergConstants.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *  http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.gravitino.iceberg.common;
-
-public class IcebergConstants {
-  public static final String CATALOG_BACKEND = "catalog-backend";
-
-  public static final String GRAVITINO_JDBC_USER = "jdbc-user";
-  public static final String ICEBERG_JDBC_USER = "jdbc.user";
-
-  public static final String GRAVITINO_JDBC_PASSWORD = "jdbc-password";
-  public static final String ICEBERG_JDBC_PASSWORD = "jdbc.password";
-  public static final String ICEBERG_JDBC_INITIALIZE = "jdbc-initialize";
-
-  public static final String GRAVITINO_JDBC_DRIVER = "jdbc-driver";
-  public static final String WAREHOUSE = "warehouse";
-  public static final String URI = "uri";
-  public static final String CATALOG_BACKEND_NAME = "catalog-backend-name";
-
-  public static final String ICEBERG_METRICS_STORE = "metricsStore";
-  public static final String ICEBERG_METRICS_STORE_RETAIN_DAYS = "metricsStoreRetainDays";
-  public static final String ICEBERG_METRICS_QUEUE_CAPACITY = "metricsQueueCapacity";
-
-  public static final String GRAVITINO_ICEBERG_REST_SERVICE_NAME = "iceberg-rest";
-}
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java
index 45bec6488de..5ff6f07a5e3 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/HiveBackendProxy.java
@@ -18,6 +18,7 @@
  */
 
 package org.apache.gravitino.iceberg.common.authentication.kerberos;
+
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
@@ -56,7 +57,7 @@ public HiveBackendProxy(
     try {
       proxyUser = UserGroupInformation.getCurrentUser();
 
-      // Replace the original client pool with org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool. Why do we need to do
+      // Replace the original client pool with IcebergHiveCachedClientPool. Why do we need to do
       // this? Because the original client pool in Iceberg uses a fixed username to create the
       // client pool, and it will not work with kerberos authentication. We need to create a new
       // client pool with the current user. For more, please see CachedClientPool#clientPool and
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java
index 6f3fa9c0214..34c0ad1dc74 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/authentication/kerberos/KerberosConfig.java
@@ -18,6 +18,7 @@
  */
 
 package org.apache.gravitino.iceberg.common.authentication.kerberos;
+
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 import org.apache.commons.lang3.StringUtils;
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java
index 6f51d813934..820914a4586 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/ops/IcebergTableOps.java
@@ -23,12 +23,12 @@
 import java.sql.DriverManager;
 import java.util.Collections;
 import java.util.Optional;
+import lombok.Getter;
+import lombok.Setter;
 import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
 import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.iceberg.common.utils.IcebergCatalogUtil;
 import org.apache.gravitino.utils.IsolatedClassLoader;
-import lombok.Getter;
-import lombok.Setter;
 import org.apache.iceberg.Transaction;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.catalog.Namespace;
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java
index ebceaf391bf..9686e443be0 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java
@@ -20,6 +20,10 @@
 
 import org.apache.gravitino.iceberg.common.ClosableHiveCatalog;
 import org.apache.gravitino.iceberg.common.IcebergConstants;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
+
 import com.google.common.base.Preconditions;
 import java.io.File;
 import java.io.IOException;
@@ -71,8 +75,8 @@ private static HiveCatalog loadHiveCatalog(Map properties) {
     } else if (authenticationConfig.isKerberosAuth()) {
       Map resultProperties = new HashMap<>(properties);
       resultProperties.put(CatalogProperties.CLIENT_POOL_CACHE_KEYS, "USER_NAME");
-      // hdfsConfiguration.set(HADOOP_SECURITY_AUTHORIZATION, "true");
-      // hdfsConfiguration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
+      hdfsConfiguration.set(HADOOP_SECURITY_AUTHORIZATION, "true");
+      hdfsConfiguration.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
       hiveCatalog.setConf(hdfsConfiguration);
       hiveCatalog.initialize(icebergCatalogName, properties);
 
diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java
index 629e9b97c5c..0acd1961118 100644
--- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java
+++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergHiveCachedClientPool.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.common.utils;/*
+/*
  * 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
@@ -17,6 +17,7 @@
  * under the License.
  *
  */
+package org.apache.gravitino.iceberg.common.utils;
 
 import com.github.benmanes.caffeine.cache.Cache;
 import com.github.benmanes.caffeine.cache.Caffeine;
@@ -51,7 +52,7 @@
  * Referred from Apache Iceberg's CachedClientPool implementation
  * hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java
  *
- * 

org.apache.gravitino.iceberg.common.utils.IcebergHiveCachedClientPool is used for every Iceberg catalog with Hive backend, I changed the + *

IcebergHiveCachedClientPool is used for every Iceberg catalog with Hive backend, I changed the * method clientPool() from * *

{@code
diff --git a/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
index 31dc27931bc..0c0b06458a3 100644
--- a/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
+++ b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/TestIcebergConfig.java
@@ -21,7 +21,6 @@
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
-import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.server.web.JettyServerConfig;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
diff --git a/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
index 65807ceec95..7a580e24d45 100644
--- a/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
+++ b/iceberg/iceberg-common/src/test/java/org/apache/gravitino/iceberg/common/utils/TestIcebergCatalogUtil.java
@@ -19,10 +19,10 @@
 
 package org.apache.gravitino.iceberg.common.utils;
 
-import org.apache.gravitino.iceberg.common.IcebergConfig;
-import org.apache.gravitino.iceberg.common.IcebergConstants;
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.iceberg.CatalogProperties;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.hive.HiveCatalog;
diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts
index 0c24fd0706e..8e6f07b25aa 100644
--- a/iceberg/iceberg-rest-server/build.gradle.kts
+++ b/iceberg/iceberg-rest-server/build.gradle.kts
@@ -31,6 +31,7 @@ val icebergVersion: String = libs.versions.iceberg.get()
 val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get()
 
 dependencies {
+  implementation(project(":catalogs:bundled-catalog", configuration = "shadow"))
   implementation(project(":core"))
   implementation(project(":common"))
   implementation(project(":iceberg:iceberg-common"))
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
index 722b0852bec..0a1bf752d9f 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/RESTService.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg;
 
 import java.util.Map;
 import javax.servlet.Servlet;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
index 839baf5324e..95c7bf91ab9 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergExceptionMapper.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service;
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
index 12f35888eb5..76fb9a66107 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapper.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service;
 
 import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.PropertyAccessor;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
index b9d4b4b8fb9..9944595ad70 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergObjectMapperProvider.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import javax.ws.rs.ext.ContextResolver;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
index 53f0ef8021d..fb0e8005c16 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/IcebergRestUtils.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service;
 
 import java.time.Instant;
 import java.time.LocalDateTime;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
index 833e18296cf..7bc482677f8 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/DummyMetricsStore.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.metrics;
 
 import java.time.Instant;
 import java.util.Map;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
index 6a0988f54b3..7294277be57 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.metrics;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
index 81e55fff6df..b50da0ca1d6 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsManager.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.metrics;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableMap;
@@ -38,7 +39,7 @@
 public class IcebergMetricsManager {
   private static final Logger LOG = LoggerFactory.getLogger(IcebergTableOps.class);
 
-  // Register org.apache.metrics.service.iceberg.gravitino.IcebergMetricsStore's short name to its full qualified class name in the map. So
+  // Register IcebergMetricsStore's short name to its full qualified class name in the map. So
   // that user doesn't need to specify the full qualified class name when creating an
   // org.apache.metrics.service.iceberg.gravitino.IcebergMetricsStore.
   private static final ImmutableMap ICEBERG_METRICS_STORE_NAMES =
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
index 03fae0ee015..2145045c750 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsStore.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.metrics;
 
 import java.io.IOException;
 import java.time.Instant;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
index 3ec38c29ea5..15fef0e3716 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergConfigOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,6 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
+
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
 import javax.servlet.http.HttpServletRequest;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
index 183ea9fa8c8..104c64029dc 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergNamespaceOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
index bc950697901..0b35f45c7d2 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
-
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import javax.inject.Inject;
diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
index 04cdbed017b..2b8585ba64a 100644
--- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
+++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/rest/IcebergTableRenameOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
 
 import com.codahale.metrics.annotation.ResponseMetered;
 import com.codahale.metrics.annotation.Timed;
diff --git a/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService b/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService
similarity index 100%
rename from iceberg/iceberg-rest-server/src/main/resources/META-INF/services/com.datastrato.gravitino.auxiliary.GravitinoAuxiliaryService
rename to iceberg/iceberg-rest-server/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
index 7a2ced56c87..1dc758a15c3 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTJdbcCatalogIT.java
@@ -18,11 +18,11 @@
  */
 package org.apache.gravitino.iceberg.integration.test;
 
-import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
-import org.apache.gravitino.iceberg.common.IcebergConfig;
-import org.apache.gravitino.iceberg.common.IcebergConstants;
 import java.util.HashMap;
 import java.util.Map;
+import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
+import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.gravitino.integration.test.container.ContainerSuite;
 import org.apache.gravitino.integration.test.container.HiveContainer;
 import org.apache.gravitino.integration.test.util.GravitinoITUtils;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
index d28cda6ea18..e562e2783e4 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceBaseIT.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.integration.test;/*
+/*
  * 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
@@ -16,8 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.integration.test;
 
-import org.apache.gravitino.iceberg.integration.test.util.IcebergRESTServerManager;
 import com.google.common.collect.ImmutableList;
 import com.google.errorprone.annotations.FormatMethod;
 import java.util.ArrayList;
@@ -30,6 +30,7 @@
 import org.apache.commons.lang3.StringUtils;
 import org.apache.gravitino.iceberg.common.IcebergCatalogBackend;
 import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.iceberg.integration.test.util.IcebergRESTServerManager;
 import org.apache.gravitino.server.web.JettyServerConfig;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SparkSession;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
index b5145b042c0..6a674200869 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/IcebergRESTServiceIT.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.integration.test;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.integration.test;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java
index 6367cb41fa9..e20be1de658 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java
@@ -20,11 +20,6 @@
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
-import org.apache.gravitino.Config;
-import org.apache.gravitino.iceberg.common.IcebergConfig;
-import org.apache.gravitino.integration.test.util.ITUtils;
-import org.apache.gravitino.rest.RESTUtils;
-import org.apache.gravitino.server.IcebergRESTServer;
 import java.io.IOException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
@@ -34,13 +29,14 @@
 import java.util.Properties;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
+import org.apache.gravitino.Config;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
+import org.apache.gravitino.integration.test.util.HttpUtils;
+import org.apache.gravitino.integration.test.util.ITUtils;
+import org.apache.gravitino.rest.RESTUtils;
+import org.apache.gravitino.server.IcebergRESTServer;
 import org.apache.gravitino.server.ServerConfig;
 import org.apache.gravitino.server.web.JettyServerConfig;
-import org.apache.hc.client5.http.classic.methods.HttpGet;
-import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
-import org.apache.hc.client5.http.impl.classic.HttpClients;
-import org.apache.hc.core5.http.ClassicHttpResponse;
-import org.apache.iceberg.rest.RESTUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -83,7 +79,7 @@ public void startIcebergRESTServer() throws Exception {
     boolean started = false;
 
     while (System.currentTimeMillis() - beginTime < 1000 * 60) {
-      started = isHttpServerUp(checkUri);
+      started = HttpUtils.isHttpServerUp(checkUri);
       if (started || (future.isPresent() && future.get().isDone())) {
         break;
       }
@@ -97,7 +93,7 @@ public void startIcebergRESTServer() throws Exception {
       } catch (Exception e) {
         throw new RuntimeException("IcebergRESTServer start failed", e);
       }
-      throw new RuntimeException("Can not start IcebergRESTServer");
+      throw new RuntimeException("Can not start IcebergRESTServer in one minute");
     }
   }
 
@@ -107,15 +103,15 @@ public void stopIcebergRESTServer() {
 
     long beginTime = System.currentTimeMillis();
     boolean started = true;
-    while (System.currentTimeMillis() - beginTime < 3 * 1000 * 60) {
+    while (System.currentTimeMillis() - beginTime < 1000 * 60) {
       sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-      started = isHttpServerUp(checkUri);
+      started = HttpUtils.isHttpServerUp(checkUri);
       if (!started) {
         break;
       }
     }
     if (started) {
-      throw new RuntimeException("Can not stop IcebergRESTServer");
+      throw new RuntimeException("Can not stop IcebergRESTServer in one minute");
     }
   }
 
@@ -152,15 +148,4 @@ private void initServerConfig() throws IOException {
     this.checkUri = String.format("http://%s:%d/metrics", host, port);
     LOG.info("Check uri:{}.", checkUri);
   }
-
-  private static boolean isHttpServerUp(String testUrl) {
-    try (CloseableHttpClient httpClient = HttpClients.createDefault()) {
-      HttpGet request = new HttpGet(testUrl);
-      ClassicHttpResponse response = httpClient.execute(request, a -> a);
-      return response.getCode() == 200;
-    } catch (Exception e) {
-      LOG.warn("Check server failed: url:{}, error message:{}", testUrl, e.getMessage());
-      return false;
-    }
-  }
 }
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
index 1c60a361617..4a52c3ee993 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java
@@ -37,8 +37,7 @@ public class IcebergRESTServerManagerForDeploy extends IcebergRESTServerManager
 
   public IcebergRESTServerManagerForDeploy() {
     String gravitinoRootDir = System.getenv("GRAVITINO_ROOT_DIR");
-    this.icebergRESTServerHome =
-        Paths.get(gravitinoRootDir, "distribution", "package", "iceberg-rest-server");
+    this.icebergRESTServerHome = Paths.get(gravitinoRootDir, "distribution", "package");
   }
 
   @Override
@@ -49,7 +48,8 @@ public Path getConfigDir() {
   @Override
   public Optional> doStartIcebergRESTServer() throws Exception {
     JdbcDriverDownloader.downloadJdbcDriver(
-        SQLITE_DRIVER_DOWNLOAD_URL, Paths.get(icebergRESTServerHome.toString(), "libs").toString());
+        SQLITE_DRIVER_DOWNLOAD_URL,
+        Paths.get(icebergRESTServerHome.toString(), "iceberg-rest-server", "libs").toString());
 
     String cmd = String.format("%s/bin/%s start", icebergRESTServerHome.toString(), SCRIPT_NAME);
     CommandExecutor.executeCommandLocalHost(
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java
index 0928e86adda..498129787a0 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java
@@ -18,7 +18,6 @@
  */
 package org.apache.gravitino.iceberg.integration.test.util;
 
-import org.apache.gravitino.server.IcebergRESTServer;
 import java.io.File;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -27,6 +26,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import org.apache.gravitino.server.IcebergRESTServer;
 
 public class IcebergRESTServerManagerForEmbedded extends IcebergRESTServerManager {
 
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
index b13441cfea0..f893ff82216 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/TestIcebergExceptionMapper.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service;
 
 import javax.ws.rs.core.Response;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
index 8abb982be4d..c30a839a7b6 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/MemoryMetricsStore.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -17,9 +17,10 @@
  * under the License.
  */
 
+package org.apache.gravitino.iceberg.service.metrics;
+
 import java.time.Instant;
 import java.util.Map;
-import org.apache.gravitino.iceberg.service.metrics.IcebergMetricsStore;
 import org.apache.iceberg.metrics.MetricsReport;
 
 /** Store Iceberg metrics in memory, used for test */
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
index 43b26816838..f02e3ecdc7c 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/metrics/TestIcebergMetricsManager.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.metrics;/*
+/*
  * 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
@@ -16,16 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
+package org.apache.gravitino.iceberg.service.metrics;
 
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.testcontainers.shaded.org.awaitility.Awaitility.await;
 
-import org.apache.gravitino.iceberg.common.IcebergConfig;
-import org.apache.gravitino.iceberg.common.IcebergConstants;
 import com.google.common.collect.ImmutableMap;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants;
+import org.apache.gravitino.iceberg.common.IcebergConfig;
 import org.apache.iceberg.metrics.ImmutableCommitMetricsResult;
 import org.apache.iceberg.metrics.ImmutableCommitReport;
 import org.apache.iceberg.metrics.MetricsReport;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
index 208664d6e7d..ed50607b97a 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergRestTestUtil.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+package org.apache.gravitino.iceberg.service.rest;
+
 import java.util.logging.Level;
 import java.util.logging.Logger;
 import org.apache.gravitino.iceberg.common.IcebergConfig;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
index 1aac961ed1f..0e7872604ec 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/IcebergTestBase.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
 
 import com.google.common.base.Joiner;
 import java.util.Map;
@@ -114,7 +115,6 @@ public Invocation.Builder getIcebergClientBuilder(
       }
     }
 
-
     return target
         .register(IcebergObjectMapperProvider.class)
         .request(MediaType.APPLICATION_JSON_TYPE)
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
index e6a6d52f863..b2f9e09efb4 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergConfig.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,6 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+package org.apache.gravitino.iceberg.service.rest;
 
 import java.util.Optional;
 import javax.ws.rs.core.Application;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
index 922c5421d44..4033dd4a46b 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergNamespaceOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
+package org.apache.gravitino.iceberg.service.rest;
 
 import com.google.common.collect.ImmutableMap;
 import java.util.Arrays;
diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
index bff250cf033..ccd60babf8f 100644
--- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
+++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/service/rest/TestIcebergTableOperations.java
@@ -1,4 +1,4 @@
-package org.apache.gravitino.iceberg.service.rest;/*
+/*
  * 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
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+package org.apache.gravitino.iceberg.service.rest;
+
 import com.google.common.collect.ImmutableSet;
 import java.util.Optional;
 import java.util.Set;
diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
index 98a7ea7e023..8dcf3bf3400 100644
--- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
+++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/MiniGravitino.java
@@ -41,7 +41,7 @@
 import org.apache.gravitino.auxiliary.AuxiliaryServiceManager;
 import org.apache.gravitino.client.HTTPClient;
 import org.apache.gravitino.client.RESTClient;
-import org.apache.gravitino.integration.test.util.AbstractIT;
+import org.apache.gravitino.integration.test.util.HttpUtils;
 import org.apache.gravitino.integration.test.util.ITUtils;
 import org.apache.gravitino.integration.test.util.KerberosProviderHelper;
 import org.apache.gravitino.integration.test.util.OAuthMockDataProvider;
@@ -156,7 +156,7 @@ public void start() throws Exception {
 
     String url = URI + "/metrics";
     while (System.currentTimeMillis() - beginTime < 1000 * 60 * 3) {
-      started = AbstractIT.isHttpServerUp(url);
+      started = HttpUtils.isHttpServerUp(url);
       if (started || future.isDone()) {
         break;
       }
@@ -187,7 +187,7 @@ public void stop() throws IOException, InterruptedException {
     String url = String.format("http://%s:%d/metrics", host, port);
     while (System.currentTimeMillis() - beginTime < 1000 * 60 * 3) {
       sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
-      started = AbstractIT.isHttpServerUp(url);
+      started = HttpUtils.isHttpServerUp(url);
       if (!started) {
         break;
       }
diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
index 7fcf5921cf2..1eabe7f58b6 100644
--- a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
+++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/AbstractIT.java
@@ -49,10 +49,6 @@
 import org.apache.gravitino.server.GravitinoServer;
 import org.apache.gravitino.server.ServerConfig;
 import org.apache.gravitino.server.web.JettyServerConfig;
-import org.apache.hc.client5.http.classic.methods.HttpGet;
-import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
-import org.apache.hc.client5.http.impl.classic.HttpClients;
-import org.apache.hc.core5.http.ClassicHttpResponse;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.extension.ExtendWith;
@@ -253,7 +249,7 @@ public static void startIntegrationTest() throws Exception {
       Awaitility.await()
           .atMost(60, TimeUnit.SECONDS)
           .pollInterval(1, TimeUnit.SECONDS)
-          .until(() -> isHttpServerUp(checkServerUrl));
+          .until(() -> HttpUtils.isHttpServerUp(checkServerUrl));
     }
 
     JettyServerConfig jettyServerConfig =
@@ -316,24 +312,4 @@ protected String readGitCommitIdFromGitFile() {
       return "";
     }
   }
-
-  /**
-   * Check if the http server is up, If http response status code is 200, then we're assuming the
-   * server is up. Or else we assume the server is not ready.
-   *
-   * 

Note: The method will ignore the response body and only check the status code. - * - * @param testUrl A url that we want to test ignore the response body. - * @return true if the server is up, false otherwise. - */ - public static boolean isHttpServerUp(String testUrl) { - try (CloseableHttpClient httpClient = HttpClients.createDefault()) { - HttpGet request = new HttpGet(testUrl); - ClassicHttpResponse response = httpClient.execute(request, a -> a); - return response.getCode() == 200; - } catch (Exception e) { - LOG.warn("Check gravitino server failed: url:{}, error message:{}", testUrl, e.getMessage()); - return false; - } - } } diff --git a/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/HttpUtils.java b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/HttpUtils.java new file mode 100644 index 00000000000..6ccac7dd76e --- /dev/null +++ b/integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/HttpUtils.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.gravitino.integration.test.util; + +import org.apache.hc.client5.http.classic.methods.HttpGet; +import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.core5.http.ClassicHttpResponse; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class HttpUtils { + private static final Logger LOG = LoggerFactory.getLogger(HttpUtils.class); + + /** + * Check if the http server is up, If http response status code is 200, then we're assuming the + * server is up. Or else we assume the server is not ready. + * + *

Note: The method will ignore the response body and only check the status code. + * + * @param testUrl A url that we want to test ignore the response body. + * @return true if the server is up, false otherwise. + */ + public static boolean isHttpServerUp(String testUrl) { + try (CloseableHttpClient httpClient = HttpClients.createDefault()) { + HttpGet request = new HttpGet(testUrl); + ClassicHttpResponse response = httpClient.execute(request, a -> a); + return response.getCode() == 200; + } catch (Exception e) { + LOG.warn("Check HTTP server failed: url:{}, error message:{}", testUrl, e.getMessage()); + return false; + } + } +} diff --git a/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java b/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java index 747aac00b18..cb6a97587f0 100644 --- a/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java +++ b/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java @@ -18,12 +18,12 @@ */ package org.apache.gravitino.server; +import java.io.File; +import java.util.Properties; import org.apache.gravitino.Config; import org.apache.gravitino.config.ConfigBuilder; import org.apache.gravitino.config.ConfigConstants; import org.apache.gravitino.config.ConfigEntry; -import java.io.File; -import java.util.Properties; public class ServerConfig extends Config { diff --git a/spark-connector/v3.3/spark/build.gradle.kts b/spark-connector/v3.3/spark/build.gradle.kts index daaf0ae2d58..1786f327c42 100644 --- a/spark-connector/v3.3/spark/build.gradle.kts +++ b/spark-connector/v3.3/spark/build.gradle.kts @@ -152,6 +152,7 @@ tasks.test { dependsOn(":catalogs:catalog-lakehouse-iceberg:jar") dependsOn(":catalogs:catalog-hive:jar") dependsOn(":iceberg:iceberg-rest-server:jar") + dependsOn(":catalogs:catalog-lakehouse-iceberg:jar") doFirst { environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.13") From 47e6b89ad4a6959fa158e09c7c0c447a25b4cd8d Mon Sep 17 00:00:00 2001 From: fanng Date: Fri, 19 Jul 2024 08:52:50 +0800 Subject: [PATCH 03/11] fix comments --- ...er.sh => gravitino-iceberg-rest-server.sh} | 8 +- build.gradle.kts | 24 ++- ...avitino-iceberg-rest-server.conf.template} | 4 +- conf/gravitino.conf.template | 2 +- .../org/apache/gravitino/GravitinoEnv.java | 165 +++++++++--------- .../auxiliary/AuxiliaryServiceManager.java | 31 +++- .../TestAuxiliaryServiceManager.java | 5 + iceberg/iceberg-rest-server/build.gradle.kts | 65 +++---- .../server/GravitinoIcebergRESTServer.java} | 13 +- .../test/util/IcebergRESTServerManager.java | 13 +- .../IcebergRESTServerManagerForDeploy.java | 2 +- .../IcebergRESTServerManagerForEmbedded.java | 8 +- 12 files changed, 185 insertions(+), 155 deletions(-) rename bin/{iceberg-rest-server.sh => gravitino-iceberg-rest-server.sh} (94%) rename conf/{iceberg-rest-server.conf.template => gravitino-iceberg-rest-server.conf.template} (94%) rename iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/{server/IcebergRESTServer.java => iceberg/server/GravitinoIcebergRESTServer.java} (87%) diff --git a/bin/iceberg-rest-server.sh b/bin/gravitino-iceberg-rest-server.sh similarity index 94% rename from bin/iceberg-rest-server.sh rename to bin/gravitino-iceberg-rest-server.sh index 15c7dec5bb9..ee89d30d2d4 100755 --- a/bin/iceberg-rest-server.sh +++ b/bin/gravitino-iceberg-rest-server.sh @@ -18,7 +18,7 @@ # under the License. # #set -ex -USAGE="-e Usage: bin/iceberg-rest-server.sh [--config ]\n\t +USAGE="-e Usage: bin/gravitino-iceberg-rest-server.sh [--config ]\n\t {start|run|stop|restart|status}" if [[ "$1" == "--config" ]]; then @@ -142,9 +142,9 @@ function stop() { } HOSTNAME=$(hostname) -GRAVITINO_OUTFILE="${GRAVITINO_LOG_DIR}/iceberg-rest-server.out" -GRAVITINO_SERVER_NAME=org.apache.gravitino.server.IcebergRESTServer -GRAVITINO_SIMPLE_SERVER_NAME=iceberg-rest-server +GRAVITINO_OUTFILE="${GRAVITINO_LOG_DIR}/gravitino-iceberg-rest-server.out" +GRAVITINO_SERVER_NAME=org.apache.gravitino.iceberg.server.GravitinoIcebergRESTServer +GRAVITINO_SIMPLE_SERVER_NAME=gravitino-iceberg-rest-server JAVA_OPTS+=" -Dfile.encoding=UTF-8" JAVA_OPTS+=" -Dlog4j2.configurationFile=file://${GRAVITINO_CONF_DIR}/log4j2.properties" diff --git a/build.gradle.kts b/build.gradle.kts index da81b276a94..0f55fbc4285 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -528,7 +528,7 @@ tasks { val outputDir = projectDir.dir("distribution") val compileDistribution by registering { - dependsOn("copySubprojectDependencies", "copyCatalogLibAndConfigs", "copySubprojectLib", "iceberg:iceberg-rest-server:copyLibs") + dependsOn("copySubprojectDependencies", "copyCatalogLibAndConfigs", "copySubprojectLib", "iceberg:iceberg-rest-server:copyLibAndConfigs") group = "gravitino distribution" outputs.dir(projectDir.dir("distribution/package")) @@ -562,18 +562,18 @@ tasks { } val compileIcebergRESTServer by registering { - dependsOn("iceberg:iceberg-rest-server:copyLibsToStandalonePackage") + dependsOn("iceberg:iceberg-rest-server:copyLibAndConfigsToStandalonePackage") group = "gravitino distribution" - outputs.dir(projectDir.dir("distribution/iceberg-rest-server")) + outputs.dir(projectDir.dir("distribution/${rootProject.name}-iceberg-rest-server")) doLast { copy { from(projectDir.dir("conf")) { - include("iceberg-rest-server.conf.template", "log4j2.properties.template") - into("iceberg-rest-server/conf") + include("${rootProject.name}-iceberg-rest-server.conf.template", "log4j2.properties.template") + into("${rootProject.name}-iceberg-rest-server/conf") } from(projectDir.dir("bin")) { - include("common.sh", "iceberg-rest-server.sh") - into("iceberg-rest-server/bin") + include("common.sh", "${rootProject.name}-iceberg-rest-server.sh") + into("${rootProject.name}-iceberg-rest-server/bin") } into(outputDir) rename { fileName -> @@ -583,10 +583,10 @@ tasks { } copy { - from(projectDir.dir("licenses")) { into("iceberg-rest-server/licenses") } - from(projectDir.file("LICENSE.bin")) { into("iceberg-rest-server") } - from(projectDir.file("NOTICE.bin")) { into("iceberg-rest-server") } - from(projectDir.file("README.md")) { into("iceberg-rest-server") } + from(projectDir.dir("licenses")) { into("${rootProject.name}-iceberg-rest-server/licenses") } + from(projectDir.file("LICENSE.bin")) { into("${rootProject.name}-iceberg-rest-server") } + from(projectDir.file("NOTICE.bin")) { into("${rootProject.name}-iceberg-rest-server") } + from(projectDir.file("README.md")) { into("${rootProject.name}-iceberg-rest-server") } into(outputDir) rename { fileName -> fileName.replace(".bin", "") @@ -619,8 +619,6 @@ tasks { val assembleIcebergRESTServer by registering(Tar::class) { dependsOn("compileIcebergRESTServer") - // fix gradlew assembleDistribution error - mustRunAfter("compileDistribution") group = "gravitino distribution" finalizedBy("checksumIcebergRESTServerDistribution") into("${rootProject.name}-iceberg-rest-server-$version-bin") diff --git a/conf/iceberg-rest-server.conf.template b/conf/gravitino-iceberg-rest-server.conf.template similarity index 94% rename from conf/iceberg-rest-server.conf.template rename to conf/gravitino-iceberg-rest-server.conf.template index b4cfddab180..64efda1e7b7 100644 --- a/conf/iceberg-rest-server.conf.template +++ b/conf/gravitino-iceberg-rest-server.conf.template @@ -40,8 +40,8 @@ gravitino.iceberg-rest.requestHeaderSize = 131072 # The response header size of the built-in web server gravitino.iceberg-rest.responseHeaderSize = 131072 -# THE CONFIGURATION FOR Iceberg backend catalog -# The backend Iceberg catalog, it's recommanded to change to hive or jdbc +# THE CONFIGURATION FOR Iceberg catalog backend +# The Iceberg catalog backend, it's recommanded to change to hive or jdbc gravitino.iceberg-rest.catalog-backend = memory # The warehouse directory of Iceberg catalog gravitino.iceberg-rest.warehouse = /tmp/ diff --git a/conf/gravitino.conf.template b/conf/gravitino.conf.template index 448f7a5973b..450a3033b49 100644 --- a/conf/gravitino.conf.template +++ b/conf/gravitino.conf.template @@ -63,7 +63,7 @@ gravitino.catalog.cache.evictionIntervalMs = 3600000 # Auxiliary service names, separate by ',' gravitino.auxService.names = iceberg-rest # Iceberg REST service classpath -gravitino.iceberg-rest.classpath = iceberg-rest-server/libs, conf +gravitino.iceberg-rest.classpath = iceberg-rest-server/libs, iceberg-rest-server/conf # Iceberg REST service host gravitino.iceberg-rest.host = 0.0.0.0 # Iceberg REST service http port diff --git a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java index 5393309277e..9d679685483 100644 --- a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java +++ b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java @@ -99,6 +99,7 @@ public class GravitinoEnv { private EventListenerManager eventListenerManager; private TagManager tagManager; + private EventBus eventBus; private GravitinoEnv() {} @@ -119,89 +120,17 @@ public static GravitinoEnv getInstance() { * Initialize the Gravitino environment. * * @param config The configuration object to initialize the environment. + * @param isGravitinoServer A boolean flag indicating whether the initialization is for the + * Gravitino server. If true, server-specific components will be initialized in addition to + * the base components. */ - public void initialize(Config config, boolean initGravitinoServerComponent) { + public void initialize(Config config, boolean isGravitinoServer) { LOG.info("Initializing Gravitino Environment..."); - this.config = config; - this.metricsSystem = new MetricsSystem(); - metricsSystem.register(new JVMMetricsSource()); - - this.eventListenerManager = new EventListenerManager(); - eventListenerManager.init( - config.getConfigsWithPrefix(EventListenerManager.GRAVITINO_EVENT_LISTENER_PREFIX)); - EventBus eventBus = eventListenerManager.createEventBus(); - - if (!initGravitinoServerComponent) { - return; + initBaseComponent(); + if (isGravitinoServer) { + initGravitinoServerComponent(); } - - // Initialize EntityStore - this.entityStore = EntityStoreFactory.createEntityStore(config); - entityStore.initialize(config); - - // create and initialize a random id generator - this.idGenerator = new RandomIdGenerator(); - - // Create and initialize metalake related modules - MetalakeManager metalakeManager = new MetalakeManager(entityStore, idGenerator); - MetalakeNormalizeDispatcher metalakeNormalizeDispatcher = - new MetalakeNormalizeDispatcher(metalakeManager); - this.metalakeDispatcher = new MetalakeEventDispatcher(eventBus, metalakeNormalizeDispatcher); - - // Create and initialize Catalog related modules - this.catalogManager = new CatalogManager(config, entityStore, idGenerator); - CatalogNormalizeDispatcher catalogNormalizeDispatcher = - new CatalogNormalizeDispatcher(catalogManager); - this.catalogDispatcher = new CatalogEventDispatcher(eventBus, catalogNormalizeDispatcher); - - SchemaOperationDispatcher schemaOperationDispatcher = - new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); - SchemaNormalizeDispatcher schemaNormalizeDispatcher = - new SchemaNormalizeDispatcher(schemaOperationDispatcher, catalogManager); - this.schemaDispatcher = new SchemaEventDispatcher(eventBus, schemaNormalizeDispatcher); - - TableOperationDispatcher tableOperationDispatcher = - new TableOperationDispatcher(catalogManager, entityStore, idGenerator); - TableNormalizeDispatcher tableNormalizeDispatcher = - new TableNormalizeDispatcher(tableOperationDispatcher, catalogManager); - this.tableDispatcher = new TableEventDispatcher(eventBus, tableNormalizeDispatcher); - - PartitionOperationDispatcher partitionOperationDispatcher = - new PartitionOperationDispatcher(catalogManager, entityStore, idGenerator); - // todo: support PartitionEventDispatcher - this.partitionDispatcher = - new PartitionNormalizeDispatcher(partitionOperationDispatcher, catalogManager); - - FilesetOperationDispatcher filesetOperationDispatcher = - new FilesetOperationDispatcher(catalogManager, entityStore, idGenerator); - FilesetNormalizeDispatcher filesetNormalizeDispatcher = - new FilesetNormalizeDispatcher(filesetOperationDispatcher, catalogManager); - this.filesetDispatcher = new FilesetEventDispatcher(eventBus, filesetNormalizeDispatcher); - - TopicOperationDispatcher topicOperationDispatcher = - new TopicOperationDispatcher(catalogManager, entityStore, idGenerator); - TopicNormalizeDispatcher topicNormalizeDispatcher = - new TopicNormalizeDispatcher(topicOperationDispatcher, catalogManager); - this.topicDispatcher = new TopicEventDispatcher(eventBus, topicNormalizeDispatcher); - - // Create and initialize access control related modules - boolean enableAuthorization = config.get(Configs.ENABLE_AUTHORIZATION); - if (enableAuthorization) { - this.accessControlManager = new AccessControlManager(entityStore, idGenerator, config); - } else { - this.accessControlManager = null; - } - - this.auxServiceManager = new AuxiliaryServiceManager(); - this.auxServiceManager.serviceInit(config); - - // Tree lock - this.lockManager = new LockManager(config); - - // Tag manager - this.tagManager = new TagManager(idGenerator, entityStore); - LOG.info("Gravitino Environment is initialized."); } @@ -362,4 +291,82 @@ public void shutdown() { LOG.info("Gravitino Environment is shut down."); } + + private void initBaseComponent() { + this.metricsSystem = new MetricsSystem(); + metricsSystem.register(new JVMMetricsSource()); + + this.eventListenerManager = new EventListenerManager(); + eventListenerManager.init( + config.getConfigsWithPrefix(EventListenerManager.GRAVITINO_EVENT_LISTENER_PREFIX)); + this.eventBus = eventListenerManager.createEventBus(); + } + + private void initGravitinoServerComponent() { + // Initialize EntityStore + this.entityStore = EntityStoreFactory.createEntityStore(config); + entityStore.initialize(config); + + // create and initialize a random id generator + this.idGenerator = new RandomIdGenerator(); + + // Create and initialize metalake related modules + MetalakeManager metalakeManager = new MetalakeManager(entityStore, idGenerator); + MetalakeNormalizeDispatcher metalakeNormalizeDispatcher = + new MetalakeNormalizeDispatcher(metalakeManager); + this.metalakeDispatcher = new MetalakeEventDispatcher(eventBus, metalakeNormalizeDispatcher); + + // Create and initialize Catalog related modules + this.catalogManager = new CatalogManager(config, entityStore, idGenerator); + CatalogNormalizeDispatcher catalogNormalizeDispatcher = + new CatalogNormalizeDispatcher(catalogManager); + this.catalogDispatcher = new CatalogEventDispatcher(eventBus, catalogNormalizeDispatcher); + + SchemaOperationDispatcher schemaOperationDispatcher = + new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + SchemaNormalizeDispatcher schemaNormalizeDispatcher = + new SchemaNormalizeDispatcher(schemaOperationDispatcher, catalogManager); + this.schemaDispatcher = new SchemaEventDispatcher(eventBus, schemaNormalizeDispatcher); + + TableOperationDispatcher tableOperationDispatcher = + new TableOperationDispatcher(catalogManager, entityStore, idGenerator); + TableNormalizeDispatcher tableNormalizeDispatcher = + new TableNormalizeDispatcher(tableOperationDispatcher, catalogManager); + this.tableDispatcher = new TableEventDispatcher(eventBus, tableNormalizeDispatcher); + + PartitionOperationDispatcher partitionOperationDispatcher = + new PartitionOperationDispatcher(catalogManager, entityStore, idGenerator); + // todo: support PartitionEventDispatcher + this.partitionDispatcher = + new PartitionNormalizeDispatcher(partitionOperationDispatcher, catalogManager); + + FilesetOperationDispatcher filesetOperationDispatcher = + new FilesetOperationDispatcher(catalogManager, entityStore, idGenerator); + FilesetNormalizeDispatcher filesetNormalizeDispatcher = + new FilesetNormalizeDispatcher(filesetOperationDispatcher, catalogManager); + this.filesetDispatcher = new FilesetEventDispatcher(eventBus, filesetNormalizeDispatcher); + + TopicOperationDispatcher topicOperationDispatcher = + new TopicOperationDispatcher(catalogManager, entityStore, idGenerator); + TopicNormalizeDispatcher topicNormalizeDispatcher = + new TopicNormalizeDispatcher(topicOperationDispatcher, catalogManager); + this.topicDispatcher = new TopicEventDispatcher(eventBus, topicNormalizeDispatcher); + + // Create and initialize access control related modules + boolean enableAuthorization = config.get(Configs.ENABLE_AUTHORIZATION); + if (enableAuthorization) { + this.accessControlManager = new AccessControlManager(entityStore, idGenerator, config); + } else { + this.accessControlManager = null; + } + + this.auxServiceManager = new AuxiliaryServiceManager(); + this.auxServiceManager.serviceInit(config); + + // Tree lock + this.lockManager = new LockManager(config); + + // Tag manager + this.tagManager = new TagManager(idGenerator, entityStore); + } } diff --git a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java index 37ebcf05085..84362facff7 100644 --- a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java +++ b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java @@ -45,6 +45,7 @@ * AuxiliaryServiceManager manage all GravitinoAuxiliaryServices with isolated classloader provided */ public class AuxiliaryServiceManager { + private static final Logger LOG = LoggerFactory.getLogger(AuxiliaryServiceManager.class); public static final String GRAVITINO_AUX_SERVICE_PREFIX = "gravitino.auxService."; public static final String AUX_SERVICE_NAMES = "names"; @@ -223,6 +224,8 @@ public void serviceStop() throws Exception { } // Extract aux service configs, transform gravitino.$serviceName.key to $serviceName.key. + // And will transform gravitino.auxService.$serviceName.key to $serviceName.key to keep + // compatibility. @VisibleForTesting static Map extractAuxiliaryServiceConfigs(Config config) { String auxServiceNames = @@ -231,21 +234,31 @@ static Map extractAuxiliaryServiceConfigs(Config config) { .getOrDefault(AUX_SERVICE_NAMES, ""); Map serviceConfigs = new HashMap<>(); serviceConfigs.put(AUX_SERVICE_NAMES, auxServiceNames); + config + .getAllConfig() + .forEach( + (k, v) -> { + if (k.startsWith(GRAVITINO_AUX_SERVICE_PREFIX)) { + serviceConfigs.put(k.substring(GRAVITINO_AUX_SERVICE_PREFIX.length()), v); + } + }); splitter .omitEmptyStrings() .trimResults() .splitToStream(auxServiceNames) .forEach( name -> - config - .getAllConfig() - .forEach( - (k, v) -> { - String prefix = "gravitino." + name + "."; - if (k.startsWith(prefix)) { - serviceConfigs.put(k.substring("gravitino.".length()), v); - } - })); + config.getAllConfig().forEach((k, v) -> extractConfig(serviceConfigs, name, k, v))); return serviceConfigs; } + + private static void extractConfig( + Map serverConfig, String serverName, String configKey, String configValue) { + if (configKey.startsWith(String.format("gravitino.%s.", serverName))) { + String extractedKey = configKey.substring("gravitino.".length()); + Preconditions.checkArgument( + !serverConfig.containsKey(extractedKey), "Duplicated configuration for " + extractedKey); + serverConfig.put(extractedKey, configValue); + } + } } diff --git a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java index 578e2f84454..94d41357540 100644 --- a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java +++ b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java @@ -38,6 +38,7 @@ public class TestAuxiliaryServiceManager { static class DummyConfig extends Config { + public static DummyConfig of(Map m) { DummyConfig dummyConfig = new DummyConfig(); dummyConfig.loadFromMap(m, k -> true); @@ -119,6 +120,8 @@ void testAuxiliaryServiceConfigs() { "a,b", "gravitino.a.test1", "test1", + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + "a.test4", + "test4", "gravitino.b.test2", "test2", "gravitino.aa.test3", @@ -133,6 +136,8 @@ void testAuxiliaryServiceConfigs() { "a,b", "a.test1", "test1", + "a.test4", + "test4", "b.test2", "test2"), serviceConfigs); diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts index 8e6f07b25aa..c09e1eec388 100644 --- a/iceberg/iceberg-rest-server/build.gradle.kts +++ b/iceberg/iceberg-rest-server/build.gradle.kts @@ -80,22 +80,30 @@ dependencies { } tasks { - val runtimeJars by registering(Copy::class) { + val copyDepends by registering(Copy::class) { from(configurations.runtimeClasspath) into("build/libs") } + jar { + finalizedBy(copyDepends) + } - val copyCatalogLibs by registering(Copy::class) { - dependsOn("jar", "runtimeJars") + register("copyLibs", Copy::class) { + dependsOn(copyDepends, "build") + from("build/libs") + into("$rootDir/distribution/package/iceberg-rest-server/libs") + } + + register("copyLibsToStandalonePackage", Copy::class) { + dependsOn(copyDepends, "build") from("build/libs") - into("$rootDir/distribution/package/catalogs/lakehouse-iceberg/libs") + into("$rootDir/distribution/gravitino-iceberg-rest-server/libs") } - val copyCatalogConfig by registering(Copy::class) { + register("copyConfigs", Copy::class) { from("src/main/resources") - into("$rootDir/distribution/package/catalogs/lakehouse-iceberg/conf") + into("$rootDir/distribution/package/iceberg-rest-server/conf") - include("lakehouse-iceberg.conf") include("core-site.xml.template") include("hdfs-site.xml.template") @@ -106,14 +114,30 @@ tasks { original } } + } + + register("copyConfigsToStandalonePackage", Copy::class) { + from("src/main/resources") + into("$rootDir/distribution/gravitino-iceberg-rest-server/conf") - exclude { details -> - details.file.isDirectory() + include("core-site.xml.template") + include("hdfs-site.xml.template") + + rename { original -> + if (original.endsWith(".template")) { + original.replace(".template", "") + } else { + original + } } } - register("copyLibAndConfig", Copy::class) { - dependsOn(copyCatalogLibs, copyCatalogConfig) + register("copyLibAndConfigs", Copy::class) { + dependsOn("copyLibs", "copyConfigs") + } + + register("copyLibAndConfigsToStandalonePackage", Copy::class) { + dependsOn("copyLibsToStandalonePackage", "copyConfigsToStandalonePackage") } } @@ -150,23 +174,4 @@ tasks.getByName("generateMetadataFileForMavenJavaPublication") { } tasks { - val copyDepends by registering(Copy::class) { - from(configurations.runtimeClasspath) - into("build/libs") - } - jar { - finalizedBy(copyDepends) - } - - register("copyLibs", Copy::class) { - dependsOn(copyDepends, "build") - from("build/libs") - into("$rootDir/distribution/package/iceberg-rest-server/libs") - } - - register("copyLibsToStandalonePackage", Copy::class) { - dependsOn(copyDepends, "build") - from("build/libs") - into("$rootDir/distribution/iceberg-rest-server/libs") - } } diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java similarity index 87% rename from iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java rename to iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java index 7853ae6d41b..364278682f0 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/server/IcebergRESTServer.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java @@ -16,28 +16,29 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.server; +package org.apache.gravitino.iceberg.server; import org.apache.gravitino.Config; import org.apache.gravitino.GravitinoEnv; import org.apache.gravitino.iceberg.RESTService; import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.server.ServerConfig; import org.apache.gravitino.server.authentication.ServerAuthenticator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class IcebergRESTServer { +public class GravitinoIcebergRESTServer { - private static final Logger LOG = LoggerFactory.getLogger(IcebergRESTServer.class); + private static final Logger LOG = LoggerFactory.getLogger(GravitinoIcebergRESTServer.class); - public static final String CONF_FILE = "iceberg-rest-server.conf"; + public static final String CONF_FILE = "gravitino-iceberg-rest-server.conf"; private final Config serverConfig; private RESTService icebergRESTService; private GravitinoEnv gravitinoEnv; - public IcebergRESTServer(Config config) { + public GravitinoIcebergRESTServer(Config config) { this.serverConfig = config; this.gravitinoEnv = GravitinoEnv.getInstance(); this.icebergRESTService = new RESTService(); @@ -67,7 +68,7 @@ public static void main(String[] args) { LOG.info("Starting Iceberg REST Server"); String confPath = System.getenv("GRAVITINO_TEST") == null ? "" : args[0]; ServerConfig serverConfig = ServerConfig.loadConfig(confPath, CONF_FILE); - IcebergRESTServer icebergRESTServer = new IcebergRESTServer(serverConfig); + GravitinoIcebergRESTServer icebergRESTServer = new GravitinoIcebergRESTServer(serverConfig); icebergRESTServer.initialize(); try { diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java index e20be1de658..6e2aa5c2852 100644 --- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java @@ -31,10 +31,10 @@ import java.util.concurrent.TimeUnit; import org.apache.gravitino.Config; import org.apache.gravitino.iceberg.common.IcebergConfig; +import org.apache.gravitino.iceberg.server.GravitinoIcebergRESTServer; import org.apache.gravitino.integration.test.util.HttpUtils; import org.apache.gravitino.integration.test.util.ITUtils; import org.apache.gravitino.rest.RESTUtils; -import org.apache.gravitino.server.IcebergRESTServer; import org.apache.gravitino.server.ServerConfig; import org.apache.gravitino.server.web.JettyServerConfig; import org.slf4j.Logger; @@ -91,9 +91,9 @@ public void startIcebergRESTServer() throws Exception { future.get().get(1, TimeUnit.SECONDS); } } catch (Exception e) { - throw new RuntimeException("IcebergRESTServer start failed", e); + throw new RuntimeException("GravitinoIcebergRESTServer start failed", e); } - throw new RuntimeException("Can not start IcebergRESTServer in one minute"); + throw new RuntimeException("Can not start GravitinoIcebergRESTServer in one minute"); } } @@ -111,7 +111,7 @@ public void stopIcebergRESTServer() { } } if (started) { - throw new RuntimeException("Can not stop IcebergRESTServer in one minute"); + throw new RuntimeException("Can not stop GravitinoIcebergRESTServer in one minute"); } } @@ -131,9 +131,10 @@ private void initServerConfig() throws IOException { Path configDir = getConfigDir(); String gravitinoRootDir = System.getenv("GRAVITINO_ROOT_DIR"); - Path configFile = Paths.get(configDir.toString(), IcebergRESTServer.CONF_FILE); + Path configFile = Paths.get(configDir.toString(), GravitinoIcebergRESTServer.CONF_FILE); customizeConfigFile( - Paths.get(gravitinoRootDir, "conf", IcebergRESTServer.CONF_FILE + ".template").toString(), + Paths.get(gravitinoRootDir, "conf", GravitinoIcebergRESTServer.CONF_FILE + ".template") + .toString(), configFile.toString()); this.serverConfig = new ServerConfig(); Properties properties = serverConfig.loadPropertiesFromFile(configFile.toFile()); diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java index 4a52c3ee993..dda2c593ef5 100644 --- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForDeploy.java @@ -30,7 +30,7 @@ public class IcebergRESTServerManagerForDeploy extends IcebergRESTServerManager { - private static final String SCRIPT_NAME = "iceberg-rest-server.sh"; + private static final String SCRIPT_NAME = "gravitino-iceberg-rest-server.sh"; private Path icebergRESTServerHome; private static final String SQLITE_DRIVER_DOWNLOAD_URL = "https://repo1.maven.org/maven2/org/xerial/sqlite-jdbc/3.42.0.0/sqlite-jdbc-3.42.0.0.jar"; diff --git a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java index 498129787a0..a198c9b41ac 100644 --- a/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManagerForEmbedded.java @@ -26,7 +26,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.gravitino.server.IcebergRESTServer; +import org.apache.gravitino.iceberg.server.GravitinoIcebergRESTServer; public class IcebergRESTServerManagerForEmbedded extends IcebergRESTServerManager { @@ -56,13 +56,13 @@ public Optional> doStartIcebergRESTServer() { executor.submit( () -> { try { - IcebergRESTServer.main( + GravitinoIcebergRESTServer.main( new String[] { - Paths.get(mockConfDir.getAbsolutePath(), IcebergRESTServer.CONF_FILE) + Paths.get(mockConfDir.getAbsolutePath(), GravitinoIcebergRESTServer.CONF_FILE) .toString() }); } catch (Exception e) { - LOG.error("Exception in startup mini IcebergRESTServer ", e); + LOG.error("Exception in startup mini GravitinoIcebergRESTServer ", e); throw new RuntimeException(e); } }); From aa0c65508b67124a7e8185bd16a0f340341141c4 Mon Sep 17 00:00:00 2001 From: fanng Date: Fri, 19 Jul 2024 19:41:19 +0800 Subject: [PATCH 04/11] xx --- bin/gravitino-iceberg-rest-server.sh | 18 +++++++++--------- .../common/utils/IcebergCatalogUtil.java | 5 ++--- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/bin/gravitino-iceberg-rest-server.sh b/bin/gravitino-iceberg-rest-server.sh index ee89d30d2d4..91079cf8e70 100755 --- a/bin/gravitino-iceberg-rest-server.sh +++ b/bin/gravitino-iceberg-rest-server.sh @@ -45,14 +45,14 @@ function check_process_status() { local pid=$(found_iceberg_rest_server_pid) if [[ -z "${pid}" ]]; then - echo "IcebergRESTServer is not running" + echo "GravitinoIcebergRESTServer is not running" else - echo "IcebergRESTServer is running[PID:$pid]" + echo "GravitinoIcebergRESTServer is running[PID:$pid]" fi } function found_iceberg_rest_server_pid() { - process_name='IcebergRESTServer'; + process_name='GravitinoIcebergRESTServer'; RUNNING_PIDS=$(ps x | grep ${process_name} | grep -v grep | awk '{print $1}'); if [[ -z "${RUNNING_PIDS}" ]]; then @@ -60,7 +60,7 @@ function found_iceberg_rest_server_pid() { fi if ! kill -0 ${RUNNING_PIDS} > /dev/null 2>&1; then - echo "IcebergRESTServer running but process is dead" + echo "GravitinoIcebergRESTServer running but process is dead" fi echo "${RUNNING_PIDS}" @@ -100,7 +100,7 @@ function start() { if [[ ! -z "${pid}" ]]; then if kill -0 ${pid} >/dev/null 2>&1; then - echo "IcebergRESTServer is already running" + echo "GravitinoIcebergRESTServer is already running" return 0; fi fi @@ -114,10 +114,10 @@ function start() { pid=$! if [[ -z "${pid}" ]]; then - echo "IcebergRESTServer start error!" + echo "GravitinoIcebergRESTServer start error!" return 1; else - echo "IcebergRESTServer start success!" + echo "GravitinoIcebergRESTServer start success!" fi sleep 2 @@ -134,10 +134,10 @@ function stop() { pid=$(found_iceberg_rest_server_pid) if [[ -z "${pid}" ]]; then - echo "IcebergRESTServer is not running" + echo "GravitinoIcebergRESTServer is not running" else wait_for_iceberg_rest_server_to_die - echo "IcebergRESTServer stop" + echo "GravitinoIcebergRESTServer stop" fi } diff --git a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java index 9686e443be0..161526df764 100644 --- a/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java +++ b/iceberg/iceberg-common/src/main/java/org/apache/gravitino/iceberg/common/utils/IcebergCatalogUtil.java @@ -18,9 +18,6 @@ */ package org.apache.gravitino.iceberg.common.utils; -import org.apache.gravitino.iceberg.common.ClosableHiveCatalog; -import org.apache.gravitino.iceberg.common.IcebergConstants; - import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION; @@ -30,6 +27,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import org.apache.gravitino.catalog.lakehouse.iceberg.IcebergConstants; +import org.apache.gravitino.iceberg.common.ClosableHiveCatalog; import org.apache.gravitino.iceberg.common.IcebergCatalogBackend; import org.apache.gravitino.iceberg.common.IcebergConfig; import org.apache.gravitino.iceberg.common.authentication.AuthenticationConfig; From 6decc4fc3f019fb5c1794bc4006199bb9ed16a23 Mon Sep 17 00:00:00 2001 From: fanng Date: Fri, 19 Jul 2024 20:59:42 +0800 Subject: [PATCH 05/11] xx --- .github/workflows/backend-integration-test.yml | 4 ++-- .github/workflows/cron-integration-test.yml | 6 ++++-- .github/workflows/spark-integration-test.yml | 5 +++-- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/.github/workflows/backend-integration-test.yml b/.github/workflows/backend-integration-test.yml index 0bf06a55d75..2464a06e1b1 100644 --- a/.github/workflows/backend-integration-test.yml +++ b/.github/workflows/backend-integration-test.yml @@ -110,8 +110,8 @@ jobs: integration-test/build/*.tar integration-test/build/trino-ci-container-log/hive/*.* integration-test/build/trino-ci-container-log/hdfs/*.* - distribution/package/logs/gravitino-server.out - distribution/package/logs/gravitino-server.log + distribution/package/logs/*.out + distribution/package/logs/*.log catalogs/**/*.log catalogs/**/*.tar distribution/**/*.log diff --git a/.github/workflows/cron-integration-test.yml b/.github/workflows/cron-integration-test.yml index 195e1b6e97b..8ae27638473 100644 --- a/.github/workflows/cron-integration-test.yml +++ b/.github/workflows/cron-integration-test.yml @@ -30,6 +30,7 @@ jobs: - core/** - dev/** - gradle/** + - iceberg/** - integration-test/** - meta/** - server/** @@ -92,12 +93,13 @@ jobs: name: integrate test reports path: | build/reports + iceberg/iceberg-rest-server/build/*.log integration-test/build/*.log integration-test/build/*.tar integration-test/build/trino-ci-container-log/hive/*.* integration-test/build/trino-ci-container-log/hdfs/*.* - distribution/package/logs/gravitino-server.out - distribution/package/logs/gravitino-server.log + distribution/package/logs/*.out + distribution/package/logs/*.log catalogs/**/*.log catalogs/**/*.tar distribution/**/*.log diff --git a/.github/workflows/spark-integration-test.yml b/.github/workflows/spark-integration-test.yml index 9f18e20afa1..a2f11061cda 100644 --- a/.github/workflows/spark-integration-test.yml +++ b/.github/workflows/spark-integration-test.yml @@ -35,6 +35,7 @@ jobs: - core/** - dev/** - gradle/** + - iceberg/** - meta/** - server/** - server-common/** @@ -105,5 +106,5 @@ jobs: spark-connector/v3.3/spark/build/spark-3.3-integration-test.log spark-connector/v3.4/spark/build/spark-3.4-integration-test.log spark-connector/v3.5/spark/build/spark-3.5-integration-test.log - distribution/package/logs/gravitino-server.out - distribution/package/logs/gravitino-server.log + distribution/package/logs/*.out + distribution/package/logs/*.log From b59a9bb0977f7ba61972238739fe45a597c9146a Mon Sep 17 00:00:00 2001 From: fanng Date: Mon, 22 Jul 2024 10:47:10 +0800 Subject: [PATCH 06/11] polish --- iceberg/iceberg-rest-server/build.gradle.kts | 5 +---- .../server/GravitinoIcebergRESTServer.java | 20 +++++++++---------- spark-connector/v3.3/spark/build.gradle.kts | 1 - 3 files changed, 11 insertions(+), 15 deletions(-) diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts index c09e1eec388..045a361d52a 100644 --- a/iceberg/iceberg-rest-server/build.gradle.kts +++ b/iceberg/iceberg-rest-server/build.gradle.kts @@ -31,7 +31,7 @@ val icebergVersion: String = libs.versions.iceberg.get() val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get() dependencies { - implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) + implementation(project(":catalogs:bundled-catalog")) implementation(project(":core")) implementation(project(":common")) implementation(project(":iceberg:iceberg-common")) @@ -172,6 +172,3 @@ tasks.clean { tasks.getByName("generateMetadataFileForMavenJavaPublication") { dependsOn("copyDepends") } - -tasks { -} diff --git a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java index 364278682f0..622f0d21acd 100644 --- a/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java @@ -44,28 +44,28 @@ public GravitinoIcebergRESTServer(Config config) { this.icebergRESTService = new RESTService(); } - public void initialize() { + private void initialize() { gravitinoEnv.initialize(serverConfig, false); icebergRESTService.serviceInit( serverConfig.getConfigsWithPrefix(IcebergConfig.ICEBERG_CONFIG_PREFIX)); ServerAuthenticator.getInstance().initialize(serverConfig); } - public void start() { + private void start() { icebergRESTService.serviceStart(); } - public void join() { + private void join() { icebergRESTService.join(); } - public void stop() throws Exception { + private void stop() throws Exception { icebergRESTService.serviceStop(); - LOG.info("Iceberg REST service stopped"); + LOG.info("Gravitino Iceberg REST service stopped"); } public static void main(String[] args) { - LOG.info("Starting Iceberg REST Server"); + LOG.info("Starting Gravitino Iceberg REST Server"); String confPath = System.getenv("GRAVITINO_TEST") == null ? "" : args[0]; ServerConfig serverConfig = ServerConfig.loadConfig(confPath, CONF_FILE); GravitinoIcebergRESTServer icebergRESTServer = new GravitinoIcebergRESTServer(serverConfig); @@ -77,7 +77,7 @@ public static void main(String[] args) { LOG.error("Error while running jettyServer", e); System.exit(-1); } - LOG.info("Done, Iceberg REST server started."); + LOG.info("Done, Gravitino Iceberg REST server started."); Runtime.getRuntime() .addShutdownHook( @@ -95,12 +95,12 @@ public static void main(String[] args) { })); icebergRESTServer.join(); - LOG.info("Shutting down Iceberg REST Server ... "); + LOG.info("Shutting down Gravitino Iceberg REST Server ... "); try { icebergRESTServer.stop(); - LOG.info("Iceberg REST Server has shut down."); + LOG.info("Gravitino Iceberg REST Server has shut down."); } catch (Exception e) { - LOG.error("Error while stopping Iceberg REST Server", e); + LOG.error("Error while stopping Gravitino Iceberg REST Server", e); } } } diff --git a/spark-connector/v3.3/spark/build.gradle.kts b/spark-connector/v3.3/spark/build.gradle.kts index 1786f327c42..daaf0ae2d58 100644 --- a/spark-connector/v3.3/spark/build.gradle.kts +++ b/spark-connector/v3.3/spark/build.gradle.kts @@ -152,7 +152,6 @@ tasks.test { dependsOn(":catalogs:catalog-lakehouse-iceberg:jar") dependsOn(":catalogs:catalog-hive:jar") dependsOn(":iceberg:iceberg-rest-server:jar") - dependsOn(":catalogs:catalog-lakehouse-iceberg:jar") doFirst { environment("GRAVITINO_CI_HIVE_DOCKER_IMAGE", "datastrato/gravitino-ci-hive:0.1.13") From 10c9012642280aa457146a070a330a209acb3f75 Mon Sep 17 00:00:00 2001 From: fanng Date: Mon, 22 Jul 2024 12:24:43 +0800 Subject: [PATCH 07/11] polish --- iceberg/iceberg-rest-server/build.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts index 045a361d52a..c2bbb0086ba 100644 --- a/iceberg/iceberg-rest-server/build.gradle.kts +++ b/iceberg/iceberg-rest-server/build.gradle.kts @@ -31,7 +31,7 @@ val icebergVersion: String = libs.versions.iceberg.get() val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get() dependencies { - implementation(project(":catalogs:bundled-catalog")) + implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) implementation(project(":core")) implementation(project(":common")) implementation(project(":iceberg:iceberg-common")) From 859f84906763260dc47b62c6be424f4637d1ab57 Mon Sep 17 00:00:00 2001 From: fanng Date: Mon, 22 Jul 2024 21:05:25 +0800 Subject: [PATCH 08/11] fix comment --- .../catalog-lakehouse-iceberg/build.gradle.kts | 1 - .../java/org/apache/gravitino/GravitinoEnv.java | 8 ++++---- .../auxiliary/AuxiliaryServiceManager.java | 7 ++++++- .../auxiliary/TestAuxiliaryServiceManager.java | 16 ++++++++++++++++ 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts index 997f70b855a..5f9aa02fcd9 100644 --- a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts +++ b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts @@ -40,7 +40,6 @@ dependencies { implementation(libs.bundles.iceberg) implementation(libs.bundles.jersey) implementation(libs.bundles.jetty) - implementation(libs.bundles.log4j) implementation(libs.cglib) implementation(libs.commons.collections4) diff --git a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java index 9d679685483..e21c6b7f6bf 100644 --- a/core/src/main/java/org/apache/gravitino/GravitinoEnv.java +++ b/core/src/main/java/org/apache/gravitino/GravitinoEnv.java @@ -127,9 +127,9 @@ public static GravitinoEnv getInstance() { public void initialize(Config config, boolean isGravitinoServer) { LOG.info("Initializing Gravitino Environment..."); this.config = config; - initBaseComponent(); + initBaseComponents(); if (isGravitinoServer) { - initGravitinoServerComponent(); + initGravitinoServerComponents(); } LOG.info("Gravitino Environment is initialized."); } @@ -292,7 +292,7 @@ public void shutdown() { LOG.info("Gravitino Environment is shut down."); } - private void initBaseComponent() { + private void initBaseComponents() { this.metricsSystem = new MetricsSystem(); metricsSystem.register(new JVMMetricsSource()); @@ -302,7 +302,7 @@ private void initBaseComponent() { this.eventBus = eventListenerManager.createEventBus(); } - private void initGravitinoServerComponent() { + private void initGravitinoServerComponents() { // Initialize EntityStore this.entityStore = EntityStoreFactory.createEntityStore(config); entityStore.initialize(config); diff --git a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java index 84362facff7..e237478215b 100644 --- a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java +++ b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java @@ -239,7 +239,12 @@ static Map extractAuxiliaryServiceConfigs(Config config) { .forEach( (k, v) -> { if (k.startsWith(GRAVITINO_AUX_SERVICE_PREFIX)) { - serviceConfigs.put(k.substring(GRAVITINO_AUX_SERVICE_PREFIX.length()), v); + String extractKey = k.substring(GRAVITINO_AUX_SERVICE_PREFIX.length()); + LOG.warn( + "The configuration {} is deprecated(still working), please use gravitino.{} instead.", + k, + extractKey); + serviceConfigs.put(extractKey, v); } }); splitter diff --git a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java index 94d41357540..d465ecf9fbd 100644 --- a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java +++ b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java @@ -141,5 +141,21 @@ void testAuxiliaryServiceConfigs() { "b.test2", "test2"), serviceConfigs); + + // Test throw exception if both new configuration and old configuration exists. + m = + ImmutableMap.of( + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + + AuxiliaryServiceManager.AUX_SERVICE_NAMES, + "a", + "gravitino.a.test1", + "test1", + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + "a.test1", + "test4"); + DummyConfig config2 = new DummyConfig(); + config2.loadFromMap(m, k -> true); + Assertions.assertThrowsExactly( + IllegalArgumentException.class, + () -> AuxiliaryServiceManager.extractAuxiliaryServiceConfigs(config2)); } } From 3cc76da6dae7f87a15d790faad7e15f90c3d0701 Mon Sep 17 00:00:00 2001 From: fanng Date: Tue, 23 Jul 2024 14:36:11 +0800 Subject: [PATCH 09/11] overwrite not throw exception --- .../gravitino/auxiliary/AuxiliaryServiceManager.java | 9 ++++++--- .../gravitino/auxiliary/TestAuxiliaryServiceManager.java | 9 +++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java index e237478215b..06db93e329d 100644 --- a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java +++ b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java @@ -261,9 +261,12 @@ private static void extractConfig( Map serverConfig, String serverName, String configKey, String configValue) { if (configKey.startsWith(String.format("gravitino.%s.", serverName))) { String extractedKey = configKey.substring("gravitino.".length()); - Preconditions.checkArgument( - !serverConfig.containsKey(extractedKey), "Duplicated configuration for " + extractedKey); - serverConfig.put(extractedKey, configValue); + String originValue = serverConfig.put(extractedKey, configValue); + if (originValue != null) { + LOG.warn( + "The configuration %s%s is overwrite by %s", + GRAVITINO_AUX_SERVICE_PREFIX, extractedKey, configKey); + } } } } diff --git a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java index d465ecf9fbd..b143cd81db8 100644 --- a/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java +++ b/core/src/test/java/org/apache/gravitino/auxiliary/TestAuxiliaryServiceManager.java @@ -142,7 +142,7 @@ void testAuxiliaryServiceConfigs() { "test2"), serviceConfigs); - // Test throw exception if both new configuration and old configuration exists. + // Test gravitino.a.test1 overwrite gravitino.auxService.a.test1 m = ImmutableMap.of( AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX @@ -154,8 +154,9 @@ void testAuxiliaryServiceConfigs() { "test4"); DummyConfig config2 = new DummyConfig(); config2.loadFromMap(m, k -> true); - Assertions.assertThrowsExactly( - IllegalArgumentException.class, - () -> AuxiliaryServiceManager.extractAuxiliaryServiceConfigs(config2)); + serviceConfigs = AuxiliaryServiceManager.extractAuxiliaryServiceConfigs(config2); + Assertions.assertEquals( + ImmutableMap.of(AuxiliaryServiceManager.AUX_SERVICE_NAMES, "a", "a.test1", "test1"), + serviceConfigs); } } From 23e5316a420e2b06367edee2a227762f021d27ae Mon Sep 17 00:00:00 2001 From: fanng Date: Tue, 23 Jul 2024 14:41:13 +0800 Subject: [PATCH 10/11] polish log --- .../apache/gravitino/auxiliary/AuxiliaryServiceManager.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java index 06db93e329d..3e37b35f200 100644 --- a/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java +++ b/core/src/main/java/org/apache/gravitino/auxiliary/AuxiliaryServiceManager.java @@ -264,8 +264,10 @@ private static void extractConfig( String originValue = serverConfig.put(extractedKey, configValue); if (originValue != null) { LOG.warn( - "The configuration %s%s is overwrite by %s", - GRAVITINO_AUX_SERVICE_PREFIX, extractedKey, configKey); + "The configuration {}{} is overwritten by {}", + GRAVITINO_AUX_SERVICE_PREFIX, + extractedKey, + configKey); } } } From 30c0bdabb233fa03250cb54520bf6eb9f8acd712 Mon Sep 17 00:00:00 2001 From: fanng Date: Tue, 23 Jul 2024 15:11:01 +0800 Subject: [PATCH 11/11] use catalog-common --- iceberg/iceberg-common/build.gradle.kts | 2 +- iceberg/iceberg-rest-server/build.gradle.kts | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/iceberg/iceberg-common/build.gradle.kts b/iceberg/iceberg-common/build.gradle.kts index 10dd4db0ab0..c27f848d61e 100644 --- a/iceberg/iceberg-common/build.gradle.kts +++ b/iceberg/iceberg-common/build.gradle.kts @@ -25,7 +25,7 @@ plugins { } dependencies { - implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) + implementation(project(":catalogs:catalog-common")) implementation(project(":core")) implementation(project(":common")) implementation(project(":server-common")) diff --git a/iceberg/iceberg-rest-server/build.gradle.kts b/iceberg/iceberg-rest-server/build.gradle.kts index c2bbb0086ba..b289c72ed03 100644 --- a/iceberg/iceberg-rest-server/build.gradle.kts +++ b/iceberg/iceberg-rest-server/build.gradle.kts @@ -31,7 +31,7 @@ val icebergVersion: String = libs.versions.iceberg.get() val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get() dependencies { - implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) + implementation(project(":catalogs:catalog-common")) implementation(project(":core")) implementation(project(":common")) implementation(project(":iceberg:iceberg-common"))