From 2353a87761e3c11e93572ebb8053a40843cf92d1 Mon Sep 17 00:00:00 2001 From: FANNG Date: Tue, 23 Jul 2024 19:05:36 +0800 Subject: [PATCH] [#2730] Split Iceberg REST service (#4005) ### What changes were proposed in this pull request? Split IcebergRESTService into one separate module, it could be managed with or without Gravitino server. #### For user 1. provide an isolated Iceberg REST server package or Gravitino package with directory `iceberg-rest-server` which contains Iceberg REST server. They both contain isolated config files and start&stop script. 2. The configuration items start with `gravitino.iceberg-rest.` are treated as Iceberg REST server configs. like `gravitino.iceberg-rest.catalog-backend`, `gravitino.iceberg-rest.httpPort`, etc. Configurations for Iceberg REST server are same in isolated package and Gravitino package. #### For developer The core is `RESTService` which start JettyServer for Iceberg REST server, * For isolated Iceberg REST server module, the entrypoint is `IcebergRESTServer` which provides configuration, metricsSystem for `RESTService`. * For GravitinoServer, `RESTService` is managed as a AuxiliaryService. ### Why are the changes needed? Iceberg rest service is managed as an auxiliary service in Gravitino server , for the users who want to use Iceberg REST service only, it introduces an extra burden. Fix: #2730 ### Does this PR introduce _any_ user-facing change? yes, will add document in https://github.com/apache/gravitino/pull/4113 ### How was this patch tested? 1. existing tests 2. test in local env as a Iceberg REST server ### related docs https://docs.google.com/document/d/1lyJwMaaJKfMqtnH9c7LwvnOHRKm7gh8Al4Sw3T1DFjM/edit --- .../workflows/backend-integration-test.yml | 6 +- .github/workflows/cron-integration-test.yml | 6 +- .github/workflows/spark-integration-test.yml | 5 +- LICENSE | 4 +- bin/gravitino-iceberg-rest-server.sh | 206 ++++++++++++++++++ bin/gravitino.sh | 2 + build.gradle.kts | 70 +++++- .../lakehouse/iceberg/IcebergConstants.java | 6 + .../build.gradle.kts | 47 +--- .../iceberg/IcebergCatalogOperations.java | 8 +- .../IcebergCatalogPropertiesMetadata.java | 5 +- .../iceberg/ops/IcebergTableOpsHelper.java | 26 +-- .../org.apache.gravitino.CatalogProvider | 2 +- .../lakehouse/iceberg/TestIcebergCatalog.java | 2 +- .../test/CatalogIcebergBaseIT.java | 4 +- .../test/CatalogIcebergKerberosHiveIT.java | 10 +- .../test/CatalogIcebergRestIT.java | 10 +- .../test/TestMultipleJDBCLoad.java | 2 +- .../iceberg/ops/TestIcebergTableUpdate.java | 5 +- ...ravitino-iceberg-rest-server.conf.template | 47 ++++ conf/gravitino.conf.template | 11 +- conf/log4j2.properties.template | 5 +- .../org/apache/gravitino/GravitinoEnv.java | 162 +++++++------- .../auxiliary/AuxiliaryServiceManager.java | 60 ++++- .../TestAuxiliaryServiceManager.java | 89 +++++++- iceberg/iceberg-common/build.gradle.kts | 88 ++++++++ .../iceberg/common}/ClosableHiveCatalog.java | 2 +- .../common}/IcebergCatalogBackend.java | 2 +- .../iceberg/common}/IcebergConfig.java | 27 +-- .../authentication/AuthenticationConfig.java | 2 +- .../kerberos/FetchFileUtils.java | 2 +- .../kerberos/HiveBackendProxy.java | 4 +- .../kerberos/KerberosClient.java | 3 +- .../kerberos/KerberosConfig.java | 4 +- .../iceberg/common}/ops/IcebergTableOps.java | 34 ++- .../common}/utils/IcebergCatalogUtil.java | 28 ++- .../utils}/IcebergHiveCachedClientPool.java | 3 +- .../iceberg/common}/TestIcebergConfig.java | 2 +- .../common}/utils/TestIcebergCatalogUtil.java | 14 +- iceberg/iceberg-rest-server/build.gradle.kts | 174 +++++++++++++++ .../apache/gravitino/iceberg/RESTService.java | 26 ++- .../server/GravitinoIcebergRESTServer.java | 106 +++++++++ .../service}/IcebergExceptionMapper.java | 3 +- .../iceberg/service}/IcebergObjectMapper.java | 3 +- .../service}/IcebergObjectMapperProvider.java | 2 +- .../iceberg/service}/IcebergRestUtils.java | 2 +- .../service}/metrics/DummyMetricsStore.java | 3 +- .../metrics/IcebergMetricsFormatter.java | 5 +- .../metrics/IcebergMetricsManager.java | 14 +- .../service}/metrics/IcebergMetricsStore.java | 3 +- .../rest/IcebergConfigOperations.java | 4 +- .../rest/IcebergNamespaceOperations.java | 6 +- .../service}/rest/IcebergTableOperations.java | 10 +- .../rest/IcebergTableRenameOperations.java | 6 +- ...vitino.auxiliary.GravitinoAuxiliaryService | 2 +- .../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 | 53 ++--- .../test/IcebergRESTServiceIT.java | 36 +-- .../test/util/IcebergRESTServerManager.java | 152 +++++++++++++ .../IcebergRESTServerManagerForDeploy.java | 68 ++++++ .../IcebergRESTServerManagerForEmbedded.java | 77 +++++++ .../service}/TestIcebergExceptionMapper.java | 2 +- .../service}/metrics/MemoryMetricsStore.java | 2 +- .../metrics/TestIcebergMetricsManager.java | 14 +- .../service}/rest/IcebergRestTestUtil.java | 12 +- .../service}/rest/IcebergTestBase.java | 15 +- .../service}/rest/TestIcebergConfig.java | 3 +- .../rest/TestIcebergNamespaceOperations.java | 3 +- .../rest/TestIcebergTableOperations.java | 2 +- .../src/test/resources/log4j2.properties | 73 +++++++ .../integration/test/MiniGravitino.java | 22 +- .../integration/test/util/AbstractIT.java | 26 +-- .../integration/test/util/HttpUtils.java | 50 +++++ 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 + 87 files changed, 1710 insertions(+), 501 deletions(-) create mode 100755 bin/gravitino-iceberg-rest-server.sh create mode 100644 conf/gravitino-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%) 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 (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/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 (87%) 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 (99%) 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 (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}/utils/TestIcebergCatalogUtil.java (83%) 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%) create mode 100644 iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java 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 (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}/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 (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/IcebergMetricsManager.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}/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%) rename {catalogs/catalog-lakehouse-iceberg => iceberg/iceberg-rest-server}/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService (92%) 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 (59%) 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 (98%) 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 (96%) 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 (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/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 create mode 100644 integration-test-common/src/test/java/org/apache/gravitino/integration/test/util/HttpUtils.java 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..2464a06e1b1 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,12 +105,13 @@ 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/*.* 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 diff --git a/LICENSE b/LICENSE index de48fa68ed5..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/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 ./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/gravitino-iceberg-rest-server.sh b/bin/gravitino-iceberg-rest-server.sh new file mode 100755 index 00000000000..91079cf8e70 --- /dev/null +++ b/bin/gravitino-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/gravitino-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 "GravitinoIcebergRESTServer is not running" + else + echo "GravitinoIcebergRESTServer is running[PID:$pid]" + fi +} + +function found_iceberg_rest_server_pid() { + process_name='GravitinoIcebergRESTServer'; + 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 "GravitinoIcebergRESTServer 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 "GravitinoIcebergRESTServer 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 "GravitinoIcebergRESTServer start error!" + return 1; + else + echo "GravitinoIcebergRESTServer 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 "GravitinoIcebergRESTServer is not running" + else + wait_for_iceberg_rest_server_to_die + echo "GravitinoIcebergRESTServer stop" + fi +} + +HOSTNAME=$(hostname) +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" +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/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/build.gradle.kts b/build.gradle.kts index 6bdf7f818ec..6a4be7dffed 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:copyLibAndConfigs") group = "gravitino distribution" outputs.dir(projectDir.dir("distribution/package")) @@ -561,8 +561,42 @@ tasks { } } + val compileIcebergRESTServer by registering { + dependsOn("iceberg:iceberg-rest-server:copyLibAndConfigsToStandalonePackage") + group = "gravitino distribution" + outputs.dir(projectDir.dir("distribution/${rootProject.name}-iceberg-rest-server")) + doLast { + copy { + from(projectDir.dir("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", "${rootProject.name}-iceberg-rest-server.sh") + into("${rootProject.name}-iceberg-rest-server/bin") + } + into(outputDir) + rename { fileName -> + fileName.replace(".template", "") + } + fileMode = 0b111101101 + } + + copy { + 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", "") + } + } + } + } + val assembleDistribution by registering(Tar::class) { - dependsOn("assembleTrinoConnector") + dependsOn("assembleTrinoConnector", "assembleIcebergRESTServer") group = "gravitino distribution" finalizedBy("checksumDistribution") into("${rootProject.name}-$version-bin") @@ -583,9 +617,36 @@ tasks { destinationDirectory.set(projectDir.dir("distribution")) } + val assembleIcebergRESTServer by registering(Tar::class) { + dependsOn("compileIcebergRESTServer") + 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 +684,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 +699,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-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 4a241c77465..5f9aa02fcd9 100644 --- a/catalogs/catalog-lakehouse-iceberg/build.gradle.kts +++ b/catalogs/catalog-lakehouse-iceberg/build.gradle.kts @@ -35,45 +35,17 @@ 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 +58,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") @@ -104,7 +69,9 @@ dependencies { exclude("org.rocksdb") } - testImplementation(libs.bundles.log4j) + testImplementation(libs.hadoop2.common) { + exclude("com.github.spotbugs") + } testImplementation(libs.jersey.test.framework.core) { exclude(group = "org.junit.jupiter") } @@ -132,6 +99,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/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..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 @@ -27,10 +27,11 @@ 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.IcebergCatalogBackend; +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..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 @@ -18,11 +18,11 @@ */ 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; @@ -36,10 +36,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..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 @@ -19,12 +19,10 @@ package org.apache.gravitino.catalog.lakehouse.iceberg.integration.test; 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.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.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-iceberg-rest-server.conf.template b/conf/gravitino-iceberg-rest-server.conf.template new file mode 100644 index 00000000000..64efda1e7b7 --- /dev/null +++ b/conf/gravitino-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 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 22fe9241553..450a3033b49 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-server/conf # 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/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..e21c6b7f6bf 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,86 +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) { + public void initialize(Config config, boolean isGravitinoServer) { LOG.info("Initializing Gravitino Environment..."); - this.config = config; - this.metricsSystem = new MetricsSystem(); - metricsSystem.register(new JVMMetricsSource()); - - // Initialize EntityStore - this.entityStore = EntityStoreFactory.createEntityStore(config); - entityStore.initialize(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 = - 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; + initBaseComponents(); + if (isGravitinoServer) { + initGravitinoServerComponents(); } - - this.auxServiceManager = new AuxiliaryServiceManager(); - this.auxServiceManager.serviceInit( - config.getConfigsWithPrefix(AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX)); - - // Tree lock - this.lockManager = new LockManager(config); - - // Tag manager - this.tagManager = new TagManager(idGenerator, entityStore); - LOG.info("Gravitino Environment is initialized."); } @@ -359,4 +291,82 @@ public void shutdown() { LOG.info("Gravitino Environment is shut down."); } + + private void initBaseComponents() { + 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 initGravitinoServerComponents() { + // 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 45012eead74..3e37b35f200 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; @@ -44,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"; @@ -124,7 +126,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 +182,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 +222,53 @@ public void serviceStop() throws Exception { throw firstException; } } + + // 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 = + config + .getConfigsWithPrefix(GRAVITINO_AUX_SERVICE_PREFIX) + .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)) { + 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 + .omitEmptyStrings() + .trimResults() + .splitToStream(auxServiceNames) + .forEach( + name -> + 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()); + String originValue = serverConfig.put(extractedKey, configValue); + if (originValue != null) { + LOG.warn( + "The configuration {}{} is overwritten by {}", + 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 2806a1dcd31..b143cd81db8 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,32 @@ 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 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 +66,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 +90,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 +110,53 @@ 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", + AuxiliaryServiceManager.GRAVITINO_AUX_SERVICE_PREFIX + "a.test4", + "test4", + "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", + "a.test4", + "test4", + "b.test2", + "test2"), + serviceConfigs); + + // Test gravitino.a.test1 overwrite gravitino.auxService.a.test1 + 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); + serviceConfigs = AuxiliaryServiceManager.extractAuxiliaryServiceConfigs(config2); + Assertions.assertEquals( + ImmutableMap.of(AuxiliaryServiceManager.AUX_SERVICE_NAMES, "a", "a.test1", "test1"), + serviceConfigs); + } } diff --git a/iceberg/iceberg-common/build.gradle.kts b/iceberg/iceberg-common/build.gradle.kts new file mode 100644 index 00000000000..c27f848d61e --- /dev/null +++ b/iceberg/iceberg-common/build.gradle.kts @@ -0,0 +1,88 @@ +/* + * 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(":catalogs:catalog-common")) + 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..bad69675494 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,14 @@ * 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.catalog.lakehouse.iceberg.IcebergConstants; import org.apache.gravitino.config.ConfigBuilder; import org.apache.gravitino.config.ConfigConstants; import org.apache.gravitino.config.ConfigEntry; @@ -32,16 +32,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 +50,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 +58,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 +66,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 +74,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 +82,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 +104,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 +112,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/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 96% 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..5ff6f07a5e3 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,7 +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; @@ -27,7 +27,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; 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..34c0ad1dc74 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,16 @@ * 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..820914a4586 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,18 +16,18 @@ * 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 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 org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Catalog; @@ -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 87% 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..161526df764 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,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.utils; +package org.apache.gravitino.iceberg.common.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; +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.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; +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; @@ -111,8 +112,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 +129,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 99% 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..0acd1961118 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 @@ -17,8 +17,7 @@ * under the License. * */ - -package org.apache.gravitino.catalog.lakehouse.iceberg; +package org.apache.gravitino.iceberg.common.utils; import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; 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 97% 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..0c0b06458a3 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,7 +17,7 @@ * 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; 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 83% 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..7a580e24d45 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 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.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; @@ -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..b289c72ed03 --- /dev/null +++ b/iceberg/iceberg-rest-server/build.gradle.kts @@ -0,0 +1,174 @@ +/* + * 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(":catalogs:catalog-common")) + 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 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/gravitino-iceberg-rest-server/libs") + } + + register("copyConfigs", Copy::class) { + from("src/main/resources") + into("$rootDir/distribution/package/iceberg-rest-server/conf") + + include("core-site.xml.template") + include("hdfs-site.xml.template") + + rename { original -> + if (original.endsWith(".template")) { + original.replace(".template", "") + } else { + original + } + } + } + + register("copyConfigsToStandalonePackage", Copy::class) { + from("src/main/resources") + into("$rootDir/distribution/gravitino-iceberg-rest-server/conf") + + include("core-site.xml.template") + include("hdfs-site.xml.template") + + rename { original -> + if (original.endsWith(".template")) { + original.replace(".template", "") + } else { + original + } + } + } + + register("copyLibAndConfigs", Copy::class) { + dependsOn("copyLibs", "copyConfigs") + } + + register("copyLibAndConfigsToStandalonePackage", Copy::class) { + dependsOn("copyLibsToStandalonePackage", "copyConfigsToStandalonePackage") + } +} + +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") +} 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..0a1bf752d9f 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 @@ -16,17 +16,17 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg; +package org.apache.gravitino.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 +39,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 +58,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 +92,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 +121,10 @@ public void serviceStop() throws Exception { icebergMetricsManager.close(); } } + + public void join() { + if (server != null) { + server.join(); + } + } } 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 new file mode 100644 index 00000000000..622f0d21acd --- /dev/null +++ b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/server/GravitinoIcebergRESTServer.java @@ -0,0 +1,106 @@ +/* + * 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.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 GravitinoIcebergRESTServer { + + private static final Logger LOG = LoggerFactory.getLogger(GravitinoIcebergRESTServer.class); + + public static final String CONF_FILE = "gravitino-iceberg-rest-server.conf"; + + private final Config serverConfig; + + private RESTService icebergRESTService; + private GravitinoEnv gravitinoEnv; + + public GravitinoIcebergRESTServer(Config config) { + this.serverConfig = config; + this.gravitinoEnv = GravitinoEnv.getInstance(); + this.icebergRESTService = new RESTService(); + } + + private void initialize() { + gravitinoEnv.initialize(serverConfig, false); + icebergRESTService.serviceInit( + serverConfig.getConfigsWithPrefix(IcebergConfig.ICEBERG_CONFIG_PREFIX)); + ServerAuthenticator.getInstance().initialize(serverConfig); + } + + private void start() { + icebergRESTService.serviceStart(); + } + + private void join() { + icebergRESTService.join(); + } + + private void stop() throws Exception { + icebergRESTService.serviceStop(); + LOG.info("Gravitino Iceberg REST service stopped"); + } + + public static void main(String[] args) { + 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); + icebergRESTServer.initialize(); + + try { + icebergRESTServer.start(); + } catch (Exception e) { + LOG.error("Error while running jettyServer", e); + System.exit(-1); + } + LOG.info("Done, Gravitino 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 Gravitino Iceberg REST Server ... "); + try { + icebergRESTServer.stop(); + LOG.info("Gravitino Iceberg REST Server has shut down."); + } catch (Exception e) { + LOG.error("Error while stopping Gravitino Iceberg REST Server", e); + } + } +} 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..95c7bf91ab9 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web; +package org.apache.gravitino.iceberg.service; 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/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..76fb9a66107 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web; +package org.apache.gravitino.iceberg.service; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.PropertyAccessor; 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 95% 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..9944595ad70 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web; +package org.apache.gravitino.iceberg.service; 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..fb0e8005c16 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web; +package org.apache.gravitino.iceberg.service; 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..7bc482677f8 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.metrics; 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/metrics/IcebergMetricsFormatter.java b/iceberg/iceberg-rest-server/src/main/java/org/apache/gravitino/iceberg/service/metrics/IcebergMetricsFormatter.java similarity index 91% 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..7294277be57 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 @@ -16,12 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.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 92% 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..b50da0ca1d6 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.metrics; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; @@ -30,22 +29,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 // 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..2145045c750 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.metrics; import java.io.IOException; import java.time.Instant; 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..15fef0e3716 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; @@ -28,7 +28,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..104c64029dc 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; @@ -34,8 +34,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..0b35f45c7d2 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; @@ -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..2b8585ba64a 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; @@ -29,8 +29,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/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService b/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService similarity index 92% rename from catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService rename to iceberg/iceberg-rest-server/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService index 543d06097fc..768b2fbd807 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService +++ b/iceberg/iceberg-rest-server/src/main/resources/META-INF/services/org.apache.gravitino.auxiliary.GravitinoAuxiliaryService @@ -16,4 +16,4 @@ # specific language governing permissions and limitations # under the License. # -org.apache.gravitino.catalog.lakehouse.iceberg.IcebergRESTService +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 59% 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..1dc758a15c3 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.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.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; @@ -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..e562e2783e4 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 @@ -16,8 +16,7 @@ * 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 com.google.common.collect.ImmutableList; import com.google.errorprone.annotations.FormatMethod; @@ -29,14 +28,10 @@ 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.iceberg.integration.test.util.IcebergRESTServerManager; 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 +45,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 +68,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 +82,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..6a674200869 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 @@ -16,20 +16,15 @@ * 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 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 +36,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 +61,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..6e2aa5c2852 --- /dev/null +++ b/iceberg/iceberg-rest-server/src/test/java/org/apache/gravitino/iceberg/integration/test/util/IcebergRESTServerManager.java @@ -0,0 +1,152 @@ +/* + * 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 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.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.ServerConfig; +import org.apache.gravitino.server.web.JettyServerConfig; +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 = HttpUtils.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("GravitinoIcebergRESTServer start failed", e); + } + throw new RuntimeException("Can not start GravitinoIcebergRESTServer in one minute"); + } + } + + public void stopIcebergRESTServer() { + doStopIcebergRESTServer(); + sleepUninterruptibly(500, TimeUnit.MILLISECONDS); + + long beginTime = System.currentTimeMillis(); + boolean started = true; + while (System.currentTimeMillis() - beginTime < 1000 * 60) { + sleepUninterruptibly(500, TimeUnit.MILLISECONDS); + started = HttpUtils.isHttpServerUp(checkUri); + if (!started) { + break; + } + } + if (started) { + throw new RuntimeException("Can not stop GravitinoIcebergRESTServer in one minute"); + } + } + + 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(), GravitinoIcebergRESTServer.CONF_FILE); + customizeConfigFile( + Paths.get(gravitinoRootDir, "conf", GravitinoIcebergRESTServer.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); + } +} 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..dda2c593ef5 --- /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 = "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"; + + public IcebergRESTServerManagerForDeploy() { + String gravitinoRootDir = System.getenv("GRAVITINO_ROOT_DIR"); + this.icebergRESTServerHome = Paths.get(gravitinoRootDir, "distribution", "package"); + } + + @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(), "iceberg-rest-server", "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..a198c9b41ac --- /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 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; +import org.apache.gravitino.iceberg.server.GravitinoIcebergRESTServer; + +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 { + GravitinoIcebergRESTServer.main( + new String[] { + Paths.get(mockConfDir.getAbsolutePath(), GravitinoIcebergRESTServer.CONF_FILE) + .toString() + }); + } catch (Exception e) { + LOG.error("Exception in startup mini GravitinoIcebergRESTServer ", 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 98% 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..f893ff82216 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 @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web; +package org.apache.gravitino.iceberg.service; 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 96% 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..c30a839a7b6 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 @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.metrics; import java.time.Instant; import java.util.Map; 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 90% 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..f02e3ecdc7c 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.metrics; +package org.apache.gravitino.iceberg.service.metrics; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.testcontainers.shaded.org.awaitility.Awaitility.await; @@ -25,7 +24,8 @@ 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.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; @@ -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..ed50607b97a 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 @@ -17,15 +17,15 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.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..0e7872604ec 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 @@ -16,11 +16,9 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.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 +27,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(); } 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..b2f9e09efb4 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import java.util.Optional; import javax.ws.rs.core.Application; 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..4033dd4a46b 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 @@ -16,8 +16,7 @@ * specific language governing permissions and limitations * under the License. */ - -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.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..ccd60babf8f 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 @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.gravitino.catalog.lakehouse.iceberg.web.rest; +package org.apache.gravitino.iceberg.service.rest; import com.google.common.collect.ImmutableSet; import java.util.Optional; 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..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; @@ -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; @@ -155,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; } @@ -186,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; } @@ -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..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: ", e); - 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/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..cb6a97587f0 100644 --- a/server/src/main/java/org/apache/gravitino/server/ServerConfig.java +++ b/server-common/src/main/java/org/apache/gravitino/server/ServerConfig.java @@ -18,6 +18,8 @@ */ 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; @@ -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")