From 9c6a75c1dde1ed60da51fb98a66306d752ce7242 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Nov 2024 17:08:34 +0100 Subject: [PATCH 01/39] Support read-only part of Iceberg REST Catalog API --- src/CMakeLists.txt | 2 + src/Common/ErrorCodes.cpp | 1 + src/Core/SettingsEnums.cpp | 3 + src/Core/SettingsEnums.h | 6 + src/Databases/Iceberg/DatabaseIceberg.cpp | 225 ++++++++++++++ src/Databases/Iceberg/DatabaseIceberg.h | 56 ++++ .../Iceberg/DatabaseIcebergSettings.cpp | 82 +++++ .../Iceberg/DatabaseIcebergSettings.h | 39 +++ src/Databases/Iceberg/ICatalog.cpp | 31 ++ src/Databases/Iceberg/ICatalog.h | 67 ++++ src/Databases/Iceberg/RestCatalog.cpp | 286 ++++++++++++++++++ src/Databases/Iceberg/RestCatalog.h | 66 ++++ src/Databases/registerDatabases.cpp | 8 + .../DataLakes/IcebergMetadata.cpp | 11 +- .../ObjectStorage/DataLakes/IcebergMetadata.h | 5 + 15 files changed, 880 insertions(+), 8 deletions(-) create mode 100644 src/Databases/Iceberg/DatabaseIceberg.cpp create mode 100644 src/Databases/Iceberg/DatabaseIceberg.h create mode 100644 src/Databases/Iceberg/DatabaseIcebergSettings.cpp create mode 100644 src/Databases/Iceberg/DatabaseIcebergSettings.h create mode 100644 src/Databases/Iceberg/ICatalog.cpp create mode 100644 src/Databases/Iceberg/ICatalog.h create mode 100644 src/Databases/Iceberg/RestCatalog.cpp create mode 100644 src/Databases/Iceberg/RestCatalog.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3627d760d4c..b9a6c49a312 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -166,6 +166,8 @@ if (TARGET ch_contrib::hdfs) add_headers_and_sources(dbms Disks/ObjectStorages/HDFS) endif() +add_headers_and_sources(dbms Databases/Iceberg) + add_headers_and_sources(dbms Disks/ObjectStorages/Cached) add_headers_and_sources(dbms Disks/ObjectStorages/Local) add_headers_and_sources(dbms Disks/ObjectStorages/Web) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 3f4a75fae3c..89523a5694f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -611,6 +611,7 @@ M(730, REFRESH_FAILED) \ M(731, QUERY_CACHE_USED_WITH_NON_THROW_OVERFLOW_MODE) \ M(733, TABLE_IS_BEING_RESTARTED) \ + M(734, ICEBERG_CATALOG_ERROR) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 89e9cb295c3..69126285eb8 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -284,4 +284,7 @@ IMPLEMENT_SETTING_ENUM( {"StochasticSimple", MergeSelectorAlgorithm::STOCHASTIC_SIMPLE}, {"Trivial", MergeSelectorAlgorithm::TRIVIAL}}) +IMPLEMENT_SETTING_ENUM(DatabaseIcebergCatalogType, ErrorCodes::BAD_ARGUMENTS, + {{"rest", DatabaseIcebergCatalogType::REST}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 35bdb8a7f65..d45b81bd8ee 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -361,4 +361,10 @@ DECLARE_SETTING_ENUM(GroupArrayActionWhenLimitReached) DECLARE_SETTING_ENUM(MergeSelectorAlgorithm) +enum class DatabaseIcebergCatalogType : uint8_t +{ + REST, +}; + +DECLARE_SETTING_ENUM(DatabaseIcebergCatalogType) } diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp new file mode 100644 index 00000000000..b81780071e9 --- /dev/null +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -0,0 +1,225 @@ +#include + +#if USE_AVRO +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include + + +namespace DB +{ +namespace DatabaseIcebergSetting +{ + extern const DatabaseIcebergSettingsString storage_endpoint; + extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; +} + +namespace +{ + std::pair parseTableName(const std::string & name) + { + auto pos = name.rfind('.'); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Table cannot have empty namespace: {}", name); + + auto table_name = name.substr(pos + 1); + auto namespace_name = name.substr(0, name.size() - table_name.size() - 1); + return {namespace_name, table_name}; + } +} + +DatabaseIceberg::DatabaseIceberg( + const std::string & database_name_, + const std::string & url_, + const DatabaseIcebergSettings & settings_, + ASTPtr database_engine_definition_) + : IDatabase(database_name_) + , url(url_) + , settings(settings_) + , database_engine_definition(database_engine_definition_) + , log(getLogger("DatabaseIceberg(" + database_name_ + ")")) +{ +} + +std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr context_) const +{ + switch (settings[DatabaseIcebergSetting::catalog_type].value) + { + case DB::DatabaseIcebergCatalogType::REST: + { + return std::make_unique(getDatabaseName(), url, context_); + } + } +} + +bool DatabaseIceberg::empty() const +{ + return getCatalog(Context::getGlobalContextInstance())->existsCatalog(); +} + +bool DatabaseIceberg::isTableExist(const String & name, ContextPtr context_) const +{ + auto [namespace_name, table_name] = parseTableName(name); + return getCatalog(context_)->existsTable(namespace_name, table_name); +} + +StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_) const +{ + auto catalog = getCatalog(context_); + auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation(); + auto [namespace_name, table_name] = parseTableName(name); + if (!catalog->tryGetTableMetadata(namespace_name, table_name, table_metadata)) + return nullptr; + + auto configuration = std::make_shared(); + ASTStorage * storage = database_engine_definition->as(); + ASTs args = storage->engine->arguments->children; + + auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) / table_metadata.getPath(); + args[0] = std::make_shared(table_endpoint.string()); + + LOG_TEST(log, "Using endpoint: {}", table_endpoint.string()); + + StorageObjectStorage::Configuration::initialize(*configuration, args, context_, false); + return std::make_shared( + configuration, + configuration->createObjectStorage(context_, /* is_readonly */ false), + context_, + StorageID(getDatabaseName(), name), + /* columns */ColumnsDescription{}, + /* constraints */ConstraintsDescription{}, + /* comment */"", + getFormatSettings(context_), + LoadingStrictnessLevel::CREATE); +} + +DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( + ContextPtr context_, + const FilterByNameFunction & /* filter_by_table_name */, + bool /* skip_not_loaded */) const +{ + Tables tables; + auto catalog = getCatalog(context_); + for (const auto & table_name : catalog->getTables()) + { + DataTypePtr type = std::make_shared(); + auto columns = ColumnsDescription{NamesAndTypesList({NameAndTypePair(std::string("a"), type)})}; + auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, ""); + tables.emplace(table_name, storage); + } + + return std::make_unique(tables, getDatabaseName()); +} + +ASTPtr DatabaseIceberg::getCreateDatabaseQuery() const +{ + const auto & create_query = std::make_shared(); + create_query->setDatabase(getDatabaseName()); + create_query->set(create_query->storage, database_engine_definition); + return create_query; +} + +ASTPtr DatabaseIceberg::getCreateTableQueryImpl( + const String & name, + ContextPtr context_, + bool /* throw_on_error */) const +{ + auto catalog = getCatalog(context_); + auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation().withSchema(); + auto [namespace_name, table_name] = parseTableName(name); + catalog->getTableMetadata(namespace_name, table_name, table_metadata); + + auto create_table_query = std::make_shared(); + auto table_storage_define = database_engine_definition->clone(); + + auto * storage = table_storage_define->as(); + storage->engine->kind = ASTFunction::Kind::TABLE_ENGINE; + storage->settings = {}; + + create_table_query->set(create_table_query->storage, table_storage_define); + + auto columns_declare_list = std::make_shared(); + auto columns_expression_list = std::make_shared(); + + columns_declare_list->set(columns_declare_list->columns, columns_expression_list); + create_table_query->set(create_table_query->columns_list, columns_declare_list); + + /// init create query. + create_table_query->setTable(name); + create_table_query->setDatabase(getDatabaseName()); + + for (const auto & column_type_and_name : table_metadata.getSchema()) + { + const auto column_declaration = std::make_shared(); + column_declaration->name = column_type_and_name.name; + column_declaration->type = makeASTDataType(column_type_and_name.type->getName()); + columns_expression_list->children.emplace_back(column_declaration); + } + + auto storage_engine_arguments = storage->engine->arguments; + if (storage_engine_arguments->children.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected number of arguments: {}", storage_engine_arguments->children.size()); + + auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) / table_metadata.getPath(); + storage_engine_arguments->children[0] = std::make_shared(table_endpoint.string()); + + return create_table_query; +} + +void registerDatabaseIceberg(DatabaseFactory & factory) +{ + auto create_fn = [](const DatabaseFactory::Arguments & args) + { + const auto * database_engine_define = args.create_query.storage; + const auto & database_engine_name = args.engine_name; + + const ASTFunction * function_define = database_engine_define->engine; + if (!function_define->arguments) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); + + ASTs & engine_args = function_define->arguments->children; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); + + const size_t max_args_num = 3; + if (engine_args.size() != max_args_num) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine must have {} arguments", max_args_num); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); + + const auto url = engine_args[0]->as()->value.safeGet(); + + DatabaseIcebergSettings database_settings; + if (database_engine_define->settings) + database_settings.loadFromQuery(*database_engine_define); + + return std::make_shared( + args.database_name, + url, + database_settings, + database_engine_define->clone()); + }; + factory.registerDatabase("Iceberg", create_fn, { .supports_arguments = true, .supports_settings = true }); +} + +} + +#endif diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h new file mode 100644 index 00000000000..1356f5aee47 --- /dev/null +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -0,0 +1,56 @@ +#pragma once +#include "config.h" + +#if USE_AVRO +#include +#include +#include + +namespace DB +{ + +class DatabaseIceberg final : public IDatabase, WithContext +{ +public: + explicit DatabaseIceberg( + const std::string & database_name_, + const std::string & url_, + const DatabaseIcebergSettings & settings_, + ASTPtr database_engine_definition_); + + String getEngineName() const override { return "Iceberg"; } + String getMetadataPath() const override { return ""; } + + bool canContainMergeTreeTables() const override { return false; } + bool canContainDistributedTables() const override { return false; } + bool shouldBeEmptyOnDetach() const override { return false; } + + bool empty() const override; + + bool isTableExist(const String & name, ContextPtr context) const override; + StoragePtr tryGetTable(const String & name, ContextPtr context) const override; + + DatabaseTablesIteratorPtr getTablesIterator( + ContextPtr context, + const FilterByNameFunction & filter_by_table_name, + bool skip_not_loaded) const override; + + void shutdown() override {} + + ASTPtr getCreateDatabaseQuery() const override; + +protected: + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; + +private: + const std::string url; + const DatabaseIcebergSettings settings; + const ASTPtr database_engine_definition; + const LoggerPtr log; + + std::unique_ptr getCatalog(ContextPtr context_) const; + +}; + +} +#endif diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp new file mode 100644 index 00000000000..d0a93edb579 --- /dev/null +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -0,0 +1,82 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; +} + +#define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE, ALIAS) \ + DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ + DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ + +#define LIST_OF_DATABASE_ICEBERG_SETTINGS(M, ALIAS) \ + DATABASE_ICEBERG_RELATED_SETTINGS(M, ALIAS) + +DECLARE_SETTINGS_TRAITS(DatabaseIcebergSettingsTraits, LIST_OF_DATABASE_ICEBERG_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(DatabaseIcebergSettingsTraits, LIST_OF_DATABASE_ICEBERG_SETTINGS) + +struct DatabaseIcebergSettingsImpl : public BaseSettings +{ +}; + +#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + DatabaseIcebergSettings##TYPE NAME = &DatabaseIcebergSettingsImpl ::NAME; + +namespace DatabaseIcebergSetting +{ +LIST_OF_DATABASE_ICEBERG_SETTINGS(INITIALIZE_SETTING_EXTERN, SKIP_ALIAS) +} + +#undef INITIALIZE_SETTING_EXTERN + +DatabaseIcebergSettings::DatabaseIcebergSettings() : impl(std::make_unique()) +{ +} + +DatabaseIcebergSettings::DatabaseIcebergSettings(const DatabaseIcebergSettings & settings) + : impl(std::make_unique(*settings.impl)) +{ +} + +DatabaseIcebergSettings::DatabaseIcebergSettings(DatabaseIcebergSettings && settings) noexcept + : impl(std::make_unique(std::move(*settings.impl))) +{ +} + +DatabaseIcebergSettings::~DatabaseIcebergSettings() = default; + +DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseIcebergSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR) + + +void DatabaseIcebergSettings::applyChanges(const SettingsChanges & changes) +{ + impl->applyChanges(changes); +} + +void DatabaseIcebergSettings::loadFromQuery(const ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + impl->applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + e.addMessage("for database engine " + storage_def.engine->name); + throw; + } + } +} + +} diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.h b/src/Databases/Iceberg/DatabaseIcebergSettings.h new file mode 100644 index 00000000000..d6908f6f3b7 --- /dev/null +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class ASTStorage; +struct DatabaseIcebergSettingsImpl; +class SettingsChanges; + +/// List of available types supported in DatabaseIcebergSettings object +#define DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ + M(CLASS_NAME, String) \ + M(CLASS_NAME, UInt64) \ + M(CLASS_NAME, DatabaseIcebergCatalogType) + +DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseIcebergSettings, DECLARE_SETTING_TRAIT) + +struct DatabaseIcebergSettings +{ + DatabaseIcebergSettings(); + DatabaseIcebergSettings(const DatabaseIcebergSettings & settings); + DatabaseIcebergSettings(DatabaseIcebergSettings && settings) noexcept; + ~DatabaseIcebergSettings(); + + DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseIcebergSettings, DECLARE_SETTING_SUBSCRIPT_OPERATOR) + + void loadFromQuery(const ASTStorage & storage_def); + + void applyChanges(const SettingsChanges & changes); + +private: + std::unique_ptr impl; +}; +} diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp new file mode 100644 index 00000000000..6ddeeeac58d --- /dev/null +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -0,0 +1,31 @@ +#include +#include + +namespace DB::ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; +} + +namespace Iceberg +{ + +std::string ICatalog::TableMetadata::getPath() const +{ + if (!with_location) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); + + if (location.starts_with("s3://")) + return location.substr(std::strlen("s3://")); + else + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported location type: {}", location); +} + +const DB::NamesAndTypesList & ICatalog::TableMetadata::getSchema() const +{ + if (!with_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); + return schema; +} + +} diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h new file mode 100644 index 00000000000..6a0412391a4 --- /dev/null +++ b/src/Databases/Iceberg/ICatalog.h @@ -0,0 +1,67 @@ +#pragma once +#include +#include + +namespace Iceberg +{ + +class ICatalog +{ +public: + using Namespace = std::string; + using Namespaces = std::vector; + using Table = std::string; + using Tables = std::vector; + + class TableMetadata; + + explicit ICatalog(const std::string & catalog_name_) : catalog_name(catalog_name_) {} + + virtual ~ICatalog() = default; + + virtual bool existsCatalog() const = 0; + + virtual Tables getTables() const = 0; + + virtual bool existsTable( + const std::string & namespace_naem, + const std::string & table_name) const = 0; + + virtual void getTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const = 0; + + virtual bool tryGetTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const = 0; + +protected: + const std::string catalog_name; +}; + +class ICatalog::TableMetadata +{ +friend class RestCatalog; + +public: + TableMetadata() = default; + + std::string getPath() const; + + const DB::NamesAndTypesList & getSchema() const; + + TableMetadata & withLocation() { with_location = true; return *this; } + TableMetadata & withSchema() { with_schema = true; return *this; } + +private: + /// starts with s3://, file://, etc + std::string location; + /// column names and types + DB::NamesAndTypesList schema; + + bool with_location = false; + bool with_schema = false; +}; +} diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp new file mode 100644 index 00000000000..e767a4cffaf --- /dev/null +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -0,0 +1,286 @@ +#include + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ + extern const int ICEBERG_CATALOG_ERROR; + extern const int LOGICAL_ERROR; +} + +namespace Iceberg +{ + +static constexpr auto namespaces_endpoint = "namespaces"; + +RestCatalog::RestCatalog( + const std::string & catalog_name_, + const std::string & base_url_, + DB::ContextPtr context_) + : ICatalog(catalog_name_) + , DB::WithContext(context_) + , base_url(base_url_) + , log(getLogger("RestCatalog(" + catalog_name_ + ")")) +{ +} + +bool RestCatalog::existsCatalog() const +{ + try + { + createReadBuffer(namespaces_endpoint)->eof(); + return true; + } + catch (...) + { + DB::tryLogCurrentException(log); + return false; + } +} + +DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer(const std::string & endpoint, const Poco::URI::QueryParameters & params) const +{ + const auto & context = getContext(); + const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + + Poco::URI url(base_url / endpoint); + if (!params.empty()) + url.setQueryParameters(params); + + return DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSkipNotFound(true) + .create(credentials); +} + +RestCatalog::Tables RestCatalog::getTables() const +{ + Namespaces namespaces; + getNamespacesRecursive("", namespaces); + + Tables tables; + for (const auto & current_namespace : namespaces) + { + auto tables_in_namespace = getTables(current_namespace); + std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); + } + return tables; +} + +void RestCatalog::getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result) const +{ + auto namespaces = getNamespaces(base_namespace); + result.reserve(result.size() + namespaces.size()); + result.insert(result.end(), namespaces.begin(), namespaces.end()); + + for (const auto & current_namespace : namespaces) + { + chassert(current_namespace.starts_with(base_namespace)); + getNamespacesRecursive(current_namespace, result); + } +} + +Poco::URI::QueryParameters RestCatalog::createParentNamespaceParams(const std::string & base_namespace) const +{ + std::vector parts; + splitInto<'.'>(parts, base_namespace); + std::string parent_param; + for (const auto & part : parts) + { + if (!parent_param.empty()) + parent_param += static_cast(0x1F); + parent_param += part; + } + return {{"parent", parent_param}}; +} + +RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_namespace) const +{ + Poco::URI::QueryParameters params; + if (!base_namespace.empty()) + params = createParentNamespaceParams(base_namespace); + + try + { + auto buf = createReadBuffer(namespaces_endpoint, params); + auto namespaces = parseNamespaces(*buf, base_namespace); + LOG_TEST(log, "Loaded {} namespaces", namespaces.size()); + return namespaces; + } + catch (const DB::HTTPException & e) + { + std::string message = fmt::format( + "Received error while fetching list of namespaces from iceberg catalog `{}`. ", + catalog_name); + + if (e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND) + message += "Namespace provided in the `parent` query parameter is not found. "; + + message += fmt::format( + "Code: {}, status: {}, message: {}", + e.code(), e.getHTTPStatus(), e.displayText()); + + throw DB::Exception(DB::ErrorCodes::ICEBERG_CATALOG_ERROR, "{}", message); + } +} + +RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const std::string & base_namespace) const +{ + if (buf.eof()) + return {}; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, buf); + + LOG_TEST(log, "Received response: {}", json_str); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); + + auto namespaces_object = object->get("namespaces").extract(); + if (!namespaces_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + + Namespaces namespaces; + for (size_t i = 0; i < namespaces_object->size(); ++i) + { + auto current_namespace_array = namespaces_object->get(static_cast(i)).extract(); + if (current_namespace_array->size() == 0) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expected namespace array to be non-empty"); + + const int current_namespace_idx = static_cast(current_namespace_array->size()) - 1; + const auto current_namespace = current_namespace_array->get(current_namespace_idx).extract(); + const auto full_namespace = base_namespace.empty() ? current_namespace : base_namespace + "." + current_namespace; + namespaces.push_back(full_namespace); + } + + return namespaces; +} + +RestCatalog::Tables RestCatalog::getTables(const Namespace & base_namespace) const +{ + const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; + auto buf = createReadBuffer(endpoint); + return parseTables(*buf, base_namespace); +} + +RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & base_namespace) const +{ + if (buf.eof()) + return {}; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, buf); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); + + auto identifiers_object = object->get("identifiers").extract(); + if (!identifiers_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + + Tables tables; + for (size_t i = 0; i < identifiers_object->size(); ++i) + { + auto current_table_json = identifiers_object->get(static_cast(i)).extract(); + auto table_name = current_table_json->get("name").extract(); + tables.push_back(base_namespace + "." + table_name); + } + return tables; +} + +bool RestCatalog::existsTable(const std::string & namespace_name, const std::string & table_name) const +{ + TableMetadata table_metadata; + return tryGetTableMetadata(namespace_name, table_name, table_metadata); +} + +bool RestCatalog::tryGetTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const +{ + try + { + return getTableMetadataImpl(namespace_name, table_name, result); + } + catch (...) + { + DB::tryLogCurrentException(log); + return false; + } +} + +void RestCatalog::getTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const +{ + if (!getTableMetadataImpl(namespace_name, table_name, result)) + throw DB::Exception(DB::ErrorCodes::ICEBERG_CATALOG_ERROR, "No response from iceberg catalog"); +} + +bool RestCatalog::getTableMetadataImpl( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const +{ + LOG_TEST(log, "Checking table {} in namespace {}", table_name, namespace_name); + + const auto endpoint = std::string(namespaces_endpoint) + "/" + namespace_name + "/tables/" + table_name; + auto buf = createReadBuffer(endpoint); + + if (buf->eof()) + { + LOG_TEST(log, "Table doesn't exist (endpoint: {})", endpoint); + return false; + } + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + + LOG_TEST(log, "Received metadata for table {}: {}", table_name, json_str); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); + + auto metadata_object = object->get("metadata").extract(); + if (!metadata_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + + if (result.with_location) + { + result.location = metadata_object->get("location").extract(); + + LOG_TEST(log, "Location for table {}: {}", table_name, result.location); + } + + if (result.with_schema) + { + int format_version = metadata_object->getValue("format-version"); + result.schema = DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first; + } + return true; +} + +} diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h new file mode 100644 index 00000000000..3b2f46b2856 --- /dev/null +++ b/src/Databases/Iceberg/RestCatalog.h @@ -0,0 +1,66 @@ +#include +#include +#include +#include +#include + +namespace DB +{ +class ReadBuffer; +} + +namespace Iceberg +{ + +class RestCatalog final : public ICatalog, private DB::WithContext +{ +public: + explicit RestCatalog( + const std::string & catalog_name_, + const std::string & base_url_, + DB::ContextPtr context_); + + ~RestCatalog() override = default; + + bool existsCatalog() const override; + + Tables getTables() const override; + + bool existsTable(const std::string & namespace_name, const std::string & table_name) const override; + + void getTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const override; + + bool tryGetTableMetadata( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const override; + +private: + const std::filesystem::path base_url; + Poco::Net::HTTPBasicCredentials credentials{}; + LoggerPtr log; + + DB::ReadWriteBufferFromHTTPPtr createReadBuffer(const std::string & endpoint, const Poco::URI::QueryParameters & params = {}) const; + + Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; + + void getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result) const; + + Namespaces getNamespaces(const Namespace & base_namespace) const; + + Namespaces parseNamespaces(DB::ReadBuffer & buf, const std::string & base_namespace) const; + + Tables getTables(const Namespace & base_namespace) const; + + Tables parseTables(DB::ReadBuffer & buf, const std::string & base_namespace) const; + + bool getTableMetadataImpl( + const std::string & namespace_name, + const std::string & table_name, + TableMetadata & result) const; +}; + +} diff --git a/src/Databases/registerDatabases.cpp b/src/Databases/registerDatabases.cpp index 4f7c229bdf4..8daba254891 100644 --- a/src/Databases/registerDatabases.cpp +++ b/src/Databases/registerDatabases.cpp @@ -36,6 +36,10 @@ void registerDatabaseS3(DatabaseFactory & factory); void registerDatabaseHDFS(DatabaseFactory & factory); #endif +#if USE_AVRO +void registerDatabaseIceberg(DatabaseFactory & factory); +#endif + void registerDatabases() { auto & factory = DatabaseFactory::instance(); @@ -68,5 +72,9 @@ void registerDatabases() #if USE_HDFS registerDatabaseHDFS(factory); #endif + +#if USE_AVRO + registerDatabaseIceberg(factory); +#endif } } diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index f0a80a41d4e..3be43daec09 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -70,9 +70,6 @@ IcebergMetadata::IcebergMetadata( { } -namespace -{ - enum class ManifestEntryStatus : uint8_t { EXISTING = 0, @@ -248,7 +245,7 @@ DataTypePtr getFieldType(const Poco::JSON::Object::Ptr & field, const String & t } -std::pair parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version, bool ignore_schema_evolution) +std::pair IcebergMetadata::parseTableSchema(const Poco::JSON::Object::Ptr & metadata_object, int format_version, bool ignore_schema_evolution) { Poco::JSON::Object::Ptr schema; Int32 current_schema_id; @@ -313,9 +310,9 @@ std::pair parseTableSchema(const Poco::JSON::Object::P for (size_t i = 0; i != fields->size(); ++i) { auto field = fields->getObject(static_cast(i)); - auto name = field->getValue("name"); + auto column_name = field->getValue("name"); bool required = field->getValue("required"); - names_and_types.push_back({name, getFieldType(field, "type", required)}); + names_and_types.push_back({column_name, getFieldType(field, "type", required)}); } return {std::move(names_and_types), current_schema_id}; @@ -380,8 +377,6 @@ std::pair getMetadataFileAndVersion( return *std::max_element(metadata_files_with_versions.begin(), metadata_files_with_versions.end()); } -} - DataLakeMetadataPtr IcebergMetadata::create(ObjectStoragePtr object_storage, ConfigurationObserverPtr configuration, ContextPtr local_context) { diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index eb5cac591f2..b28de471b40 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -96,6 +96,11 @@ public: static DataLakeMetadataPtr create(ObjectStoragePtr object_storage, ConfigurationObserverPtr configuration, ContextPtr local_context); + static std::pair parseTableSchema( + const Poco::JSON::Object::Ptr & metadata_object, + int format_version, + bool ignore_schema_evolution); + private: size_t getVersion() const { return metadata_version; } From fb28c1645361fec0405797ac547a9c5a4bdcfea8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Nov 2024 20:42:50 +0100 Subject: [PATCH 02/39] Unfinished test --- .../docker_compose_iceberg_rest_catalog.yml | 27 ++++ .../compose/docker_compose_minio.yml | 4 + tests/integration/helpers/cluster.py | 16 +++ .../test_database_iceberg/__init__.py | 0 .../integration/test_database_iceberg/test.py | 118 ++++++++++++++++++ 5 files changed, 165 insertions(+) create mode 100644 tests/integration/compose/docker_compose_iceberg_rest_catalog.yml create mode 100644 tests/integration/test_database_iceberg/__init__.py create mode 100644 tests/integration/test_database_iceberg/test.py diff --git a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml new file mode 100644 index 00000000000..7c786b84fc1 --- /dev/null +++ b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml @@ -0,0 +1,27 @@ +services: + rest: + image: tabulario/iceberg-rest + container_name: iceberg-rest + ports: + - 8181:8181 + environment: + - AWS_ACCESS_KEY_ID=minio + - AWS_SECRET_ACCESS_KEY=minio123 + - AWS_REGION=us-east-1 + - CATALOG_WAREHOUSE=s3://warehouse/ + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9001 + + spark-iceberg: + image: tabulario/spark-iceberg + container_name: spark-iceberg + depends_on: + - rest + environment: + - AWS_ACCESS_KEY_ID=minio + - AWS_SECRET_ACCESS_KEY=minio123 + - AWS_REGION=us-east-1 + ports: + - 8080:8080 + - 10000:10000 + - 10001:10001 diff --git a/tests/integration/compose/docker_compose_minio.yml b/tests/integration/compose/docker_compose_minio.yml index 7fbe3796a0c..b0dad3d1ab8 100644 --- a/tests/integration/compose/docker_compose_minio.yml +++ b/tests/integration/compose/docker_compose_minio.yml @@ -14,6 +14,10 @@ services: depends_on: - proxy1 - proxy2 + networks: + default: + aliases: + - warehouse.minio # HTTP proxies for Minio. proxy1: diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6751f205fb8..56659c1eb88 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -440,6 +440,7 @@ class ClickHouseCluster: zookeeper_keyfile=None, zookeeper_certfile=None, with_spark=False, + with_iceberg_catalog=False, ): for param in list(os.environ.keys()): logging.debug("ENV %40s %s" % (param, os.environ[param])) @@ -1462,6 +1463,18 @@ class ClickHouseCluster: ) return self.base_minio_cmd + def setup_iceberg_catalog_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.base_cmd.extend( + ["--file", p.join(docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml")] + ) + self.base_iceberg_catalog_cmd = self.compose_cmd( + "--env-file", + instance.env_file, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml"), + ) + return self.base_minio_cmd + def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_azurite = True env_variables["AZURITE_PORT"] = str(self.azurite_port) @@ -1917,6 +1930,9 @@ class ClickHouseCluster: cmds.append( self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir) ) + cmds.append( + self.setup_iceberg_catalog_cmd(instance, env_variables, docker_compose_yml_dir) + ) if with_azurite and not self.with_azurite: cmds.append( diff --git a/tests/integration/test_database_iceberg/__init__.py b/tests/integration/test_database_iceberg/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py new file mode 100644 index 00000000000..dc1e75ad40d --- /dev/null +++ b/tests/integration/test_database_iceberg/test.py @@ -0,0 +1,118 @@ +import glob +import json +import logging +import os +import time +import uuid +import requests +import pytest + +from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm +from helpers.s3_tools import ( + get_file_contents, + list_s3_objects, + prepare_s3_bucket, +) + +BASE_URL = "http://rest:8181/v1" +BASE_URL_LOCAL = "http://localhost:8181/v1" + + +def create_namespace(name): + payload = { + "namespace": [name], + "properties": {"owner": "clickhouse", "description": "test namespace"}, + } + + headers = {"Content-Type": "application/json"} + response = requests.post( + f"{BASE_URL_LOCAL}/namespaces", headers=headers, data=json.dumps(payload) + ) + if response.status_code == 200: + print(f"Namespace '{name}' created successfully.") + else: + raise Exception( + f"Failed to create namespace. Status code: {response.status_code}, Response: {response.text}" + ) + + +def list_namespaces(): + response = requests.get(f"{BASE_URL_LOCAL}/namespaces") + if response.status_code == 200: + return response.json() + else: + raise Exception(f"Failed to list namespaces: {response.status_code}") + + +def create_table(name, namespace): + payload = { + "name": name, + "schema": { + "type": "struct", + "fields": [ + {"id": 1, "name": "name", "type": "String", "required": True}, + {"id": 2, "name": "age", "type": "Int", "required": False}, + ], + }, + } + + headers = {"Content-Type": "application/json"} + response = requests.post( + f"{BASE_URL_LOCAL}/namespaces/{namespace}/tables", + headers=headers, + data=json.dumps(payload), + ) + if response.status_code == 200: + print(f"Table '{name}' created successfully.") + else: + raise Exception( + f"Failed to create a table. Status code: {response.status_code}, Response: {response.text}" + ) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=[], + user_configs=[], + with_minio=True, + stay_alive=True, + ) + + logging.info("Starting cluster...") + cluster.start() + + cluster.minio_client.make_bucket("warehouse") + prepare_s3_bucket(cluster) + + yield cluster + + finally: + cluster.shutdown() + + +def test_simple(started_cluster): + # TODO: properly wait for container + time.sleep(10) + + namespace = "kssenii.test.namespace" + root_namespace = "kssenii" + + create_namespace(namespace) + assert root_namespace in list_namespaces()["namespaces"][0][0] + + node = started_cluster.instances["node1"] + node.query( + f""" +CREATE DATABASE demo ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') +SETTINGS catalog_type = 'rest', storage_endpoint = 'http://{started_cluster.minio_ip}:{started_cluster.minio_port}/' + """ + ) + + table_name = "testtable" + create_table(table_name, "kssenii") + + assert namespace in node.query("USE demo; SHOW TABLES") From 1c3fcfa355e73957436490a3007019d29c201627 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Nov 2024 20:57:43 +0100 Subject: [PATCH 03/39] Fix style check --- src/Databases/Iceberg/DatabaseIceberg.cpp | 7 ++++++- src/Databases/Iceberg/RestCatalog.h | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index b81780071e9..a84e4fe1cbe 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -31,13 +31,18 @@ namespace DatabaseIcebergSetting extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; } +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + namespace { std::pair parseTableName(const std::string & name) { auto pos = name.rfind('.'); if (pos == std::string::npos) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Table cannot have empty namespace: {}", name); + throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Table cannot have empty namespace: {}", name); auto table_name = name.substr(pos + 1); auto namespace_name = name.substr(0, name.size() - table_name.size() - 1); diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 3b2f46b2856..b33d5accda4 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -1,3 +1,4 @@ +#pragma once #include #include #include From ca3dfe5d8ee416cc0487aca50e87def0e4fb0de3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Nov 2024 20:05:08 +0000 Subject: [PATCH 04/39] Automatic style fix --- tests/integration/helpers/cluster.py | 15 ++++++++++++--- tests/integration/test_database_iceberg/test.py | 9 +++------ 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 56659c1eb88..78e9c581629 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1463,9 +1463,16 @@ class ClickHouseCluster: ) return self.base_minio_cmd - def setup_iceberg_catalog_cmd(self, instance, env_variables, docker_compose_yml_dir): + def setup_iceberg_catalog_cmd( + self, instance, env_variables, docker_compose_yml_dir + ): self.base_cmd.extend( - ["--file", p.join(docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml")] + [ + "--file", + p.join( + docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml" + ), + ] ) self.base_iceberg_catalog_cmd = self.compose_cmd( "--env-file", @@ -1931,7 +1938,9 @@ class ClickHouseCluster: self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir) ) cmds.append( - self.setup_iceberg_catalog_cmd(instance, env_variables, docker_compose_yml_dir) + self.setup_iceberg_catalog_cmd( + instance, env_variables, docker_compose_yml_dir + ) ) if with_azurite and not self.with_azurite: diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index dc1e75ad40d..85b3fef6eb8 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -4,15 +4,12 @@ import logging import os import time import uuid -import requests + import pytest +import requests from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm -from helpers.s3_tools import ( - get_file_contents, - list_s3_objects, - prepare_s3_bucket, -) +from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_bucket BASE_URL = "http://rest:8181/v1" BASE_URL_LOCAL = "http://localhost:8181/v1" From e565f5f1c7cea4e4a6406b24827feef47c058e6c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Nov 2024 16:41:39 +0100 Subject: [PATCH 05/39] Add a test --- .../docker_compose_iceberg_rest_catalog.yml | 54 +++++++-- tests/integration/helpers/cluster.py | 10 +- .../integration/test_database_iceberg/test.py | 103 ++++++++++++++++-- 3 files changed, 144 insertions(+), 23 deletions(-) diff --git a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml index 7c786b84fc1..0a4be37335f 100644 --- a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml +++ b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml @@ -1,7 +1,21 @@ services: + spark-iceberg: + image: tabulario/spark-iceberg + container_name: spark-iceberg + build: spark/ + depends_on: + - rest + - minio + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + ports: + - 8080:8080 + - 10000:10000 + - 10001:10001 rest: image: tabulario/iceberg-rest - container_name: iceberg-rest ports: - 8181:8181 environment: @@ -10,18 +24,36 @@ services: - AWS_REGION=us-east-1 - CATALOG_WAREHOUSE=s3://warehouse/ - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO - - CATALOG_S3_ENDPOINT=http://minio:9001 - - spark-iceberg: - image: tabulario/spark-iceberg - container_name: spark-iceberg + - CATALOG_S3_ENDPOINT=http://minio:9000 + minio: + image: minio/minio + container_name: minio + environment: + - MINIO_ROOT_USER=minio + - MINIO_ROOT_PASSWORD=minio123 + - MINIO_DOMAIN=minio + networks: + default: + aliases: + - warehouse.minio + ports: + - 9001:9001 + - 9000:9000 + command: ["server", "/data", "--console-address", ":9001"] + mc: depends_on: - - rest + - minio + image: minio/mc + container_name: mc environment: - AWS_ACCESS_KEY_ID=minio - AWS_SECRET_ACCESS_KEY=minio123 - AWS_REGION=us-east-1 - ports: - - 8080:8080 - - 10000:10000 - - 10001:10001 + entrypoint: > + /bin/sh -c " + until (/usr/bin/mc config host add minio http://minio:9000 minio minio123) do echo '...waiting...' && sleep 1; done; + /usr/bin/mc rm -r --force minio/warehouse; + /usr/bin/mc mb minio/warehouse --ignore-existing; + /usr/bin/mc policy set public minio/warehouse; + tail -f /dev/null + " diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 78e9c581629..517d0e36865 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -440,7 +440,6 @@ class ClickHouseCluster: zookeeper_keyfile=None, zookeeper_certfile=None, with_spark=False, - with_iceberg_catalog=False, ): for param in list(os.environ.keys()): logging.debug("ENV %40s %s" % (param, os.environ[param])) @@ -569,6 +568,7 @@ class ClickHouseCluster: self.resolver_logs_dir = os.path.join(self.instances_dir, "resolver") self.spark_session = None + self.with_iceberg_catalog = False self.with_azurite = False self.azurite_container = "azurite-container" @@ -1480,7 +1480,8 @@ class ClickHouseCluster: "--file", p.join(docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml"), ) - return self.base_minio_cmd + return self.base_iceberg_catalog_cmd + #return self.base_minio_cmd def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_azurite = True @@ -1648,6 +1649,7 @@ class ClickHouseCluster: with_hive=False, with_coredns=False, with_prometheus=False, + with_iceberg_catalog=False, handle_prometheus_remote_write=False, handle_prometheus_remote_read=False, use_old_analyzer=None, @@ -1750,6 +1752,7 @@ class ClickHouseCluster: with_coredns=with_coredns, with_cassandra=with_cassandra, with_ldap=with_ldap, + with_iceberg_catalog=with_iceberg_catalog, use_old_analyzer=use_old_analyzer, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, @@ -1937,6 +1940,8 @@ class ClickHouseCluster: cmds.append( self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir) ) + + if with_iceberg_catalog and not self.with_iceberg_catalog: cmds.append( self.setup_iceberg_catalog_cmd( instance, env_variables, docker_compose_yml_dir @@ -3383,6 +3388,7 @@ class ClickHouseInstance: with_coredns, with_cassandra, with_ldap, + with_iceberg_catalog, use_old_analyzer, server_bin_path, odbc_bridge_bin_path, diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 85b3fef6eb8..cc72b50a93c 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -7,12 +7,31 @@ import uuid import pytest import requests +from minio import Minio +import urllib3 + +from pyiceberg.catalog import load_catalog +from pyiceberg.schema import Schema +from pyiceberg.types import ( + TimestampType, + FloatType, + DoubleType, + StringType, + NestedField, + StructType, +) +from pyiceberg.partitioning import PartitionSpec, PartitionField +from pyiceberg.transforms import DayTransform +from pyiceberg.table.sorting import SortOrder, SortField +from pyiceberg.transforms import IdentityTransform + from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_bucket BASE_URL = "http://rest:8181/v1" BASE_URL_LOCAL = "http://localhost:8181/v1" +BASE_URL_LOCAL_RAW = "http://localhost:8181" def create_namespace(name): @@ -44,6 +63,7 @@ def list_namespaces(): def create_table(name, namespace): payload = { "name": name, + "location": "s3://warehouse/", "schema": { "type": "struct", "fields": [ @@ -70,20 +90,19 @@ def create_table(name, namespace): @pytest.fixture(scope="module") def started_cluster(): try: - cluster = ClickHouseCluster(__file__, with_spark=True) + cluster = ClickHouseCluster(__file__) cluster.add_instance( "node1", main_configs=[], user_configs=[], - with_minio=True, stay_alive=True, + with_iceberg_catalog=True, ) logging.info("Starting cluster...") cluster.start() - cluster.minio_client.make_bucket("warehouse") - prepare_s3_bucket(cluster) + # prepare_s3_bucket(cluster) yield cluster @@ -95,10 +114,10 @@ def test_simple(started_cluster): # TODO: properly wait for container time.sleep(10) - namespace = "kssenii.test.namespace" - root_namespace = "kssenii" + namespace_1 = "clickhouse.test.A" + root_namespace = "clickhouse" - create_namespace(namespace) + create_namespace(namespace_1) assert root_namespace in list_namespaces()["namespaces"][0][0] node = started_cluster.instances["node1"] @@ -109,7 +128,71 @@ SETTINGS catalog_type = 'rest', storage_endpoint = 'http://{started_cluster.mini """ ) - table_name = "testtable" - create_table(table_name, "kssenii") + catalog_name = "demo" - assert namespace in node.query("USE demo; SHOW TABLES") + catalog = load_catalog( + "demo", + **{ + "uri": BASE_URL_LOCAL_RAW, + "type": "rest", + "s3.endpoint": f"http://minio:9000", + "s3.access-key-id": "minio", + "s3.secret-access-key": "minio123", + }, + ) + namespace_2 = "clickhouse.test.B" + catalog.create_namespace(namespace_2) + + assert [(root_namespace,)] == catalog.list_namespaces() + + tables = catalog.list_tables(namespace_2) + assert len(tables) == 0 + + schema = Schema( + NestedField( + field_id=1, name="datetime", field_type=TimestampType(), required=True + ), + NestedField(field_id=2, name="symbol", field_type=StringType(), required=True), + NestedField(field_id=3, name="bid", field_type=FloatType(), required=False), + NestedField(field_id=4, name="ask", field_type=DoubleType(), required=False), + NestedField( + field_id=5, + name="details", + field_type=StructType( + NestedField( + field_id=4, + name="created_by", + field_type=StringType(), + required=False, + ), + ), + required=False, + ), + ) + + partition_spec = PartitionSpec( + PartitionField( + source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day" + ) + ) + + sort_order = SortOrder(SortField(source_id=2, transform=IdentityTransform())) + + for table in ["tableA", "tableB"]: + catalog.create_table( + identifier=f"{namespace_2}.{table}", + schema=schema, + location=f"s3://warehouse", + partition_spec=partition_spec, + sort_order=sort_order, + ) + + def check(): + assert f"{namespace_2}.tableA\n{namespace_2}.tableB\n" == node.query("SELECT name FROM system.tables WHERE database = 'demo' ORDER BY name") + + expected = "CREATE TABLE demo.`clickhouse.test.B.tableA`\\n(\\n `datetime` DateTime64(6),\\n `symbol` String,\\n `bid` Nullable(Float32),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://None:9001/warehouse\\', \\'minio\\', \\'[HIDDEN]\\')\n"; + assert expected == node.query(f"SHOW CREATE TABLE demo.`{namespace_2}.tableA`") + + check() + node.restart_clickhouse() + check() From f8255fb4ae94888c828640d828e1f5c526239ab0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Nov 2024 18:17:10 +0100 Subject: [PATCH 06/39] One more test --- .../integration/test_database_iceberg/test.py | 245 ++++++++++-------- 1 file changed, 130 insertions(+), 115 deletions(-) diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index cc72b50a93c..a1f4ad6ad06 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -10,6 +10,7 @@ import requests from minio import Minio import urllib3 +from helpers.test_tools import TSV, csv_compare from pyiceberg.catalog import load_catalog from pyiceberg.schema import Schema from pyiceberg.types import ( @@ -33,23 +34,33 @@ BASE_URL = "http://rest:8181/v1" BASE_URL_LOCAL = "http://localhost:8181/v1" BASE_URL_LOCAL_RAW = "http://localhost:8181" +CATALOG_NAME = "demo" -def create_namespace(name): - payload = { - "namespace": [name], - "properties": {"owner": "clickhouse", "description": "test namespace"}, - } - - headers = {"Content-Type": "application/json"} - response = requests.post( - f"{BASE_URL_LOCAL}/namespaces", headers=headers, data=json.dumps(payload) +DEFAULT_SCHEMA = Schema( + NestedField(field_id=1, name="datetime", field_type=TimestampType(), required=True), + NestedField(field_id=2, name="symbol", field_type=StringType(), required=True), + NestedField(field_id=3, name="bid", field_type=FloatType(), required=False), + NestedField(field_id=4, name="ask", field_type=DoubleType(), required=False), + NestedField( + field_id=5, + name="details", + field_type=StructType( + NestedField( + field_id=4, + name="created_by", + field_type=StringType(), + required=False, + ), + ), + required=False, + ), +) +DEFAULT_PARTITION_SPEC = PartitionSpec( + PartitionField( + source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day" ) - if response.status_code == 200: - print(f"Namespace '{name}' created successfully.") - else: - raise Exception( - f"Failed to create namespace. Status code: {response.status_code}, Response: {response.text}" - ) +) +DEFAULT_SORT_ORDER = SortOrder(SortField(source_id=2, transform=IdentityTransform())) def list_namespaces(): @@ -60,31 +71,44 @@ def list_namespaces(): raise Exception(f"Failed to list namespaces: {response.status_code}") -def create_table(name, namespace): - payload = { - "name": name, - "location": "s3://warehouse/", - "schema": { - "type": "struct", - "fields": [ - {"id": 1, "name": "name", "type": "String", "required": True}, - {"id": 2, "name": "age", "type": "Int", "required": False}, - ], +def load_catalog_impl(): + return load_catalog( + CATALOG_NAME, + **{ + "uri": BASE_URL_LOCAL_RAW, + "type": "rest", + "s3.endpoint": f"http://minio:9000", + "s3.access-key-id": "minio", + "s3.secret-access-key": "minio123", }, - } - - headers = {"Content-Type": "application/json"} - response = requests.post( - f"{BASE_URL_LOCAL}/namespaces/{namespace}/tables", - headers=headers, - data=json.dumps(payload), ) - if response.status_code == 200: - print(f"Table '{name}' created successfully.") - else: - raise Exception( - f"Failed to create a table. Status code: {response.status_code}, Response: {response.text}" - ) + + +def create_table( + catalog, + namespace, + table, + schema=DEFAULT_SCHEMA, + partition_spec=DEFAULT_PARTITION_SPEC, + sort_order=DEFAULT_SORT_ORDER, +): + catalog.create_table( + identifier=f"{namespace}.{table}", + schema=schema, + location=f"s3://warehouse", + partition_spec=partition_spec, + sort_order=sort_order, + ) + + +def create_clickhouse_iceberg_database(started_cluster, node, name): + node.query( + f""" +DROP DATABASE IF EXISTS {name}; +CREATE DATABASE {name} ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') +SETTINGS catalog_type = 'rest', storage_endpoint = 'http://{started_cluster.minio_ip}:{started_cluster.minio_port}/' + """ + ) @pytest.fixture(scope="module") @@ -102,7 +126,8 @@ def started_cluster(): logging.info("Starting cluster...") cluster.start() - # prepare_s3_bucket(cluster) + # TODO: properly wait for container + time.sleep(10) yield cluster @@ -111,88 +136,78 @@ def started_cluster(): def test_simple(started_cluster): - # TODO: properly wait for container - time.sleep(10) - - namespace_1 = "clickhouse.test.A" - root_namespace = "clickhouse" - - create_namespace(namespace_1) - assert root_namespace in list_namespaces()["namespaces"][0][0] - node = started_cluster.instances["node1"] - node.query( - f""" -CREATE DATABASE demo ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') -SETTINGS catalog_type = 'rest', storage_endpoint = 'http://{started_cluster.minio_ip}:{started_cluster.minio_port}/' - """ - ) - catalog_name = "demo" + root_namespace = "clickhouse" + namespace_1 = "clickhouse.testA.A" + namespace_2 = "clickhouse.testB.B" + namespace_1_tables = ["tableA", "tableB"] + namespace_2_tables = ["tableC", "tableD"] - catalog = load_catalog( - "demo", - **{ - "uri": BASE_URL_LOCAL_RAW, - "type": "rest", - "s3.endpoint": f"http://minio:9000", - "s3.access-key-id": "minio", - "s3.secret-access-key": "minio123", - }, - ) - namespace_2 = "clickhouse.test.B" - catalog.create_namespace(namespace_2) + catalog = load_catalog_impl() + for namespace in [namespace_1, namespace_2]: + catalog.create_namespace(namespace) + + assert root_namespace in list_namespaces()["namespaces"][0][0] assert [(root_namespace,)] == catalog.list_namespaces() - tables = catalog.list_tables(namespace_2) - assert len(tables) == 0 + for namespace in [namespace_1, namespace_2]: + assert len(catalog.list_tables(namespace)) == 0 - schema = Schema( - NestedField( - field_id=1, name="datetime", field_type=TimestampType(), required=True - ), - NestedField(field_id=2, name="symbol", field_type=StringType(), required=True), - NestedField(field_id=3, name="bid", field_type=FloatType(), required=False), - NestedField(field_id=4, name="ask", field_type=DoubleType(), required=False), - NestedField( - field_id=5, - name="details", - field_type=StructType( - NestedField( - field_id=4, - name="created_by", - field_type=StringType(), - required=False, - ), - ), - required=False, - ), + create_clickhouse_iceberg_database(started_cluster, node, CATALOG_NAME) + + tables_list = "" + for table in namespace_1_tables: + create_table(catalog, namespace_1, table) + if len(tables_list) > 0: + tables_list += "\n" + tables_list += f"{namespace_1}.{table}" + + for table in namespace_2_tables: + create_table(catalog, namespace_2, table) + if len(tables_list) > 0: + tables_list += "\n" + tables_list += f"{namespace_2}.{table}" + + assert ( + tables_list + == node.query( + f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' ORDER BY name" + ).strip() ) - - partition_spec = PartitionSpec( - PartitionField( - source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day" - ) - ) - - sort_order = SortOrder(SortField(source_id=2, transform=IdentityTransform())) - - for table in ["tableA", "tableB"]: - catalog.create_table( - identifier=f"{namespace_2}.{table}", - schema=schema, - location=f"s3://warehouse", - partition_spec=partition_spec, - sort_order=sort_order, - ) - - def check(): - assert f"{namespace_2}.tableA\n{namespace_2}.tableB\n" == node.query("SELECT name FROM system.tables WHERE database = 'demo' ORDER BY name") - - expected = "CREATE TABLE demo.`clickhouse.test.B.tableA`\\n(\\n `datetime` DateTime64(6),\\n `symbol` String,\\n `bid` Nullable(Float32),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://None:9001/warehouse\\', \\'minio\\', \\'[HIDDEN]\\')\n"; - assert expected == node.query(f"SHOW CREATE TABLE demo.`{namespace_2}.tableA`") - - check() node.restart_clickhouse() - check() + assert ( + tables_list + == node.query( + f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' ORDER BY name" + ).strip() + ) + + expected = f"CREATE TABLE {CATALOG_NAME}.`{namespace_2}.tableC`\\n(\\n `datetime` DateTime64(6),\\n `symbol` String,\\n `bid` Nullable(Float32),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://None:9001/warehouse\\', \\'minio\\', \\'[HIDDEN]\\')\n" + assert expected == node.query( + f"SHOW CREATE TABLE {CATALOG_NAME}.`{namespace_2}.tableC`" + ) + + +def test_different_namespaces(started_cluster): + node = started_cluster.instances["node1"] + namespaces = ["A", "A.B.C", "A.B.C.D", "A.B.C.D.E", "A.B.C.D.E.F", "A.B.C.D.E.FF", "B", "B.C", "B.CC"] + tables = ["A", "B", "C", "D", "E", "F"] + catalog = load_catalog_impl() + + for namespace in namespaces: + #if namespace in catalog.list_namespaces()["namesoaces"]: + # catalog.drop_namespace(namespace) + catalog.create_namespace(namespace) + for table in tables: + create_table(catalog, namespace, table) + + create_clickhouse_iceberg_database(started_cluster, node, CATALOG_NAME) + + for namespace in namespaces: + for table in tables: + table_name = f"{namespace}.{table}" + assert int(node.query( + f"SELECT count() FROM system.tables WHERE database = '{CATALOG_NAME}' and name = '{table_name}'" + )) From 19854ff78ae84b0f8c03a8e7df8406ed3a37371d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Nov 2024 17:24:25 +0000 Subject: [PATCH 07/39] Automatic style fix --- tests/integration/helpers/cluster.py | 2 +- .../integration/test_database_iceberg/test.py | 41 +++++++++++-------- 2 files changed, 26 insertions(+), 17 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 517d0e36865..0d0cf585763 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1481,7 +1481,7 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_iceberg_rest_catalog.yml"), ) return self.base_iceberg_catalog_cmd - #return self.base_minio_cmd + # return self.base_minio_cmd def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_azurite = True diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index a1f4ad6ad06..43686694a60 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -7,28 +7,25 @@ import uuid import pytest import requests -from minio import Minio import urllib3 - -from helpers.test_tools import TSV, csv_compare +from minio import Minio from pyiceberg.catalog import load_catalog +from pyiceberg.partitioning import PartitionField, PartitionSpec from pyiceberg.schema import Schema +from pyiceberg.table.sorting import SortField, SortOrder +from pyiceberg.transforms import DayTransform, IdentityTransform from pyiceberg.types import ( - TimestampType, - FloatType, DoubleType, - StringType, + FloatType, NestedField, + StringType, StructType, + TimestampType, ) -from pyiceberg.partitioning import PartitionSpec, PartitionField -from pyiceberg.transforms import DayTransform -from pyiceberg.table.sorting import SortOrder, SortField -from pyiceberg.transforms import IdentityTransform - from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_bucket +from helpers.test_tools import TSV, csv_compare BASE_URL = "http://rest:8181/v1" BASE_URL_LOCAL = "http://localhost:8181/v1" @@ -192,12 +189,22 @@ def test_simple(started_cluster): def test_different_namespaces(started_cluster): node = started_cluster.instances["node1"] - namespaces = ["A", "A.B.C", "A.B.C.D", "A.B.C.D.E", "A.B.C.D.E.F", "A.B.C.D.E.FF", "B", "B.C", "B.CC"] + namespaces = [ + "A", + "A.B.C", + "A.B.C.D", + "A.B.C.D.E", + "A.B.C.D.E.F", + "A.B.C.D.E.FF", + "B", + "B.C", + "B.CC", + ] tables = ["A", "B", "C", "D", "E", "F"] catalog = load_catalog_impl() for namespace in namespaces: - #if namespace in catalog.list_namespaces()["namesoaces"]: + # if namespace in catalog.list_namespaces()["namesoaces"]: # catalog.drop_namespace(namespace) catalog.create_namespace(namespace) for table in tables: @@ -208,6 +215,8 @@ def test_different_namespaces(started_cluster): for namespace in namespaces: for table in tables: table_name = f"{namespace}.{table}" - assert int(node.query( - f"SELECT count() FROM system.tables WHERE database = '{CATALOG_NAME}' and name = '{table_name}'" - )) + assert int( + node.query( + f"SELECT count() FROM system.tables WHERE database = '{CATALOG_NAME}' and name = '{table_name}'" + ) + ) From f8fbd0330c8a2a9c485bbf8a389d9b330ed76e3f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 12 Nov 2024 19:04:22 +0100 Subject: [PATCH 08/39] Update requirements.txt --- docker/test/integration/runner/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index bb0c4d001e6..bccf15b8495 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -101,3 +101,4 @@ wadllib==1.3.6 websocket-client==1.8.0 wheel==0.38.1 zipp==1.0.0 +pyiceberg==0.7.1 From 1e8ee2034a0aa2d23b5256e8cbb1d5c80ca07ab4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Nov 2024 15:26:32 +0100 Subject: [PATCH 09/39] Fix test run with runner --- docker/test/integration/runner/requirements.txt | 4 ++-- .../compose/docker_compose_iceberg_rest_catalog.yml | 7 +++---- tests/integration/test_database_iceberg/test.py | 5 +++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index bccf15b8495..9e3f88e1350 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -36,7 +36,7 @@ geomet==0.2.1.post1 grpcio-tools==1.60.0 grpcio==1.60.0 gssapi==1.8.3 -httplib2==0.20.2 +httplib2==0.22.0 idna==3.7 importlib-metadata==4.6.4 iniconfig==2.0.0 @@ -72,7 +72,7 @@ pyarrow==17.0.0 pycparser==2.22 pycryptodome==3.20.0 pymongo==3.11.0 -pyparsing==2.4.7 +pyparsing==3.1.0 pyspark==3.3.2 pyspnego==0.10.2 pytest-order==1.0.0 diff --git a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml index 0a4be37335f..b529bf7d3ff 100644 --- a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml +++ b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml @@ -17,7 +17,7 @@ services: rest: image: tabulario/iceberg-rest ports: - - 8181:8181 + - 8182:8181 environment: - AWS_ACCESS_KEY_ID=minio - AWS_SECRET_ACCESS_KEY=minio123 @@ -36,9 +36,8 @@ services: default: aliases: - warehouse.minio - ports: - - 9001:9001 - - 9000:9000 + expose: + - 9001 command: ["server", "/data", "--console-address", ":9001"] mc: depends_on: diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 43686694a60..05344b7467c 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -28,8 +28,8 @@ from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_buck from helpers.test_tools import TSV, csv_compare BASE_URL = "http://rest:8181/v1" -BASE_URL_LOCAL = "http://localhost:8181/v1" -BASE_URL_LOCAL_RAW = "http://localhost:8181" +BASE_URL_LOCAL = "http://localhost:8182/v1" +BASE_URL_LOCAL_RAW = "http://localhost:8182" CATALOG_NAME = "demo" @@ -125,6 +125,7 @@ def started_cluster(): # TODO: properly wait for container time.sleep(10) + #cluster.minio_client.make_bucket("warehouse") yield cluster From 6dfd4ad942e648ce215e5dc8ab32445b50c45fc3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Nov 2024 19:58:04 +0100 Subject: [PATCH 10/39] Better --- src/Core/SettingsEnums.cpp | 7 ++ src/Core/SettingsEnums.h | 11 ++ src/Databases/Iceberg/DatabaseIceberg.cpp | 63 ++++++++--- src/Databases/Iceberg/DatabaseIceberg.h | 3 +- .../Iceberg/DatabaseIcebergSettings.cpp | 1 + .../Iceberg/DatabaseIcebergSettings.h | 3 +- src/Databases/Iceberg/ICatalog.cpp | 2 +- src/Databases/Iceberg/ICatalog.h | 2 + src/Databases/Iceberg/RestCatalog.cpp | 47 ++++++-- src/Databases/Iceberg/RestCatalog.h | 9 +- .../docker_compose_iceberg_rest_catalog.yml | 7 +- .../integration/test_database_iceberg/test.py | 105 +++++++++++++++--- 12 files changed, 213 insertions(+), 47 deletions(-) diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 69126285eb8..19581fb2f93 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -287,4 +287,11 @@ IMPLEMENT_SETTING_ENUM( IMPLEMENT_SETTING_ENUM(DatabaseIcebergCatalogType, ErrorCodes::BAD_ARGUMENTS, {{"rest", DatabaseIcebergCatalogType::REST}}) +IMPLEMENT_SETTING_ENUM(DatabaseIcebergStorageType, ErrorCodes::BAD_ARGUMENTS, + {{"s3", DatabaseIcebergStorageType::S3}, + {"azure", DatabaseIcebergStorageType::Azure}, + {"hdfs", DatabaseIcebergStorageType::HDFS}, + {"local", DatabaseIcebergStorageType::Local}, + }) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index d45b81bd8ee..004e2aec789 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -367,4 +367,15 @@ enum class DatabaseIcebergCatalogType : uint8_t }; DECLARE_SETTING_ENUM(DatabaseIcebergCatalogType) + +enum class DatabaseIcebergStorageType : uint8_t +{ + S3, + Azure, + Local, + HDFS, +}; + +DECLARE_SETTING_ENUM(DatabaseIcebergStorageType) + } diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index a84e4fe1cbe..7efde3a5671 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include #include @@ -29,6 +28,7 @@ namespace DatabaseIcebergSetting { extern const DatabaseIcebergSettingsString storage_endpoint; extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; + extern const DatabaseIcebergSettingsDatabaseIcebergStorageType storage_type; } namespace ErrorCodes @@ -38,6 +38,10 @@ namespace ErrorCodes namespace { + /// Parse a string, containing at least one dot, into a two substrings: + /// A.B.C.D.E -> A.B.C.D and E, where + /// `A.B.C.D` is a table "namespace". + /// `E` is a table name. std::pair parseTableName(const std::string & name) { auto pos = name.rfind('.'); @@ -74,41 +78,73 @@ std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr contex } } +std::shared_ptr DatabaseIceberg::getConfiguration() const +{ + switch (settings[DatabaseIcebergSetting::storage_type].value) + { + case DB::DatabaseIcebergStorageType::S3: + { + return std::make_shared(); + } + case DB::DatabaseIcebergStorageType::Azure: + { + return std::make_shared(); + } + case DB::DatabaseIcebergStorageType::HDFS: + { + return std::make_shared(); + } + case DB::DatabaseIcebergStorageType::Local: + { + return std::make_shared(); + } + } +} + bool DatabaseIceberg::empty() const { - return getCatalog(Context::getGlobalContextInstance())->existsCatalog(); + return getCatalog(Context::getGlobalContextInstance())->empty(); } bool DatabaseIceberg::isTableExist(const String & name, ContextPtr context_) const { - auto [namespace_name, table_name] = parseTableName(name); + const auto [namespace_name, table_name] = parseTableName(name); return getCatalog(context_)->existsTable(namespace_name, table_name); } StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_) const { auto catalog = getCatalog(context_); - auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation(); + auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation().withSchema(); auto [namespace_name, table_name] = parseTableName(name); + if (!catalog->tryGetTableMetadata(namespace_name, table_name, table_metadata)) return nullptr; - auto configuration = std::make_shared(); + /// Take database engine definition AST as base. ASTStorage * storage = database_engine_definition->as(); ASTs args = storage->engine->arguments->children; - auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) / table_metadata.getPath(); + /// Replace Iceberg Catalog endpoint with storage path endpoint of requested table. + auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) + / table_metadata.getPath() + / ""; + args[0] = std::make_shared(table_endpoint.string()); - LOG_TEST(log, "Using endpoint: {}", table_endpoint.string()); + LOG_TEST(log, "Using table endpoint: {}", table_endpoint.string()); + + const auto columns = ColumnsDescription(table_metadata.getSchema()); + const auto configuration = getConfiguration(); + /// with_table_structure = false: because there will be no table stucture in table definition AST. + StorageObjectStorage::Configuration::initialize(*configuration, args, context_, /* with_table_structure */false); - StorageObjectStorage::Configuration::initialize(*configuration, args, context_, false); return std::make_shared( configuration, configuration->createObjectStorage(context_, /* is_readonly */ false), context_, StorageID(getDatabaseName(), name), - /* columns */ColumnsDescription{}, + /* columns */columns, /* constraints */ConstraintsDescription{}, /* comment */"", getFormatSettings(context_), @@ -117,16 +153,17 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( ContextPtr context_, - const FilterByNameFunction & /* filter_by_table_name */, + const FilterByNameFunction & filter_by_table_name, bool /* skip_not_loaded */) const { Tables tables; auto catalog = getCatalog(context_); for (const auto & table_name : catalog->getTables()) { - DataTypePtr type = std::make_shared(); - auto columns = ColumnsDescription{NamesAndTypesList({NameAndTypePair(std::string("a"), type)})}; - auto storage = std::make_shared(StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, ""); + if (filter_by_table_name && !filter_by_table_name(table_name)) + continue; + + auto storage = tryGetTable(table_name, context_); tables.emplace(table_name, storage); } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 1356f5aee47..41ed17f58d0 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -49,7 +50,7 @@ private: const LoggerPtr log; std::unique_ptr getCatalog(ContextPtr context_) const; - + std::shared_ptr getConfiguration() const; }; } diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index d0a93edb579..8383c372a52 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes #define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE, ALIAS) \ DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ + DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ #define LIST_OF_DATABASE_ICEBERG_SETTINGS(M, ALIAS) \ diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.h b/src/Databases/Iceberg/DatabaseIcebergSettings.h index d6908f6f3b7..5d9d120efed 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.h +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.h @@ -16,7 +16,8 @@ class SettingsChanges; #define DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ M(CLASS_NAME, String) \ M(CLASS_NAME, UInt64) \ - M(CLASS_NAME, DatabaseIcebergCatalogType) + M(CLASS_NAME, DatabaseIcebergCatalogType) \ + M(CLASS_NAME, DatabaseIcebergStorageType) DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseIcebergSettings, DECLARE_SETTING_TRAIT) diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index 6ddeeeac58d..a67d88fc555 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -18,7 +18,7 @@ std::string ICatalog::TableMetadata::getPath() const if (location.starts_with("s3://")) return location.substr(std::strlen("s3://")); else - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported location type: {}", location); + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); } const DB::NamesAndTypesList & ICatalog::TableMetadata::getSchema() const diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index 6a0412391a4..a9e8c24b01b 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -21,6 +21,8 @@ public: virtual bool existsCatalog() const = 0; + virtual bool empty() const = 0; + virtual Tables getTables() const = 0; virtual bool existsTable( diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index e767a4cffaf..86abec72c29 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -52,6 +52,30 @@ bool RestCatalog::existsCatalog() const } } +bool RestCatalog::empty() const +{ + try + { + bool found_table = false; + auto stop_condition = [&](const std::string & namespace_name) -> bool + { + const auto tables = getTables(namespace_name, /* limit */1); + found_table = !tables.empty(); + return found_table; + }; + + Namespaces namespaces; + getNamespacesRecursive("", namespaces, stop_condition); + + return found_table; + } + catch (...) + { + DB::tryLogCurrentException(log); + return true; + } +} + DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer(const std::string & endpoint, const Poco::URI::QueryParameters & params) const { const auto & context = getContext(); @@ -73,7 +97,7 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer(const std::string & RestCatalog::Tables RestCatalog::getTables() const { Namespaces namespaces; - getNamespacesRecursive("", namespaces); + getNamespacesRecursive("", namespaces, {}); Tables tables; for (const auto & current_namespace : namespaces) @@ -84,7 +108,7 @@ RestCatalog::Tables RestCatalog::getTables() const return tables; } -void RestCatalog::getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result) const +void RestCatalog::getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result, StopCondition stop_condition) const { auto namespaces = getNamespaces(base_namespace); result.reserve(result.size() + namespaces.size()); @@ -93,7 +117,11 @@ void RestCatalog::getNamespacesRecursive(const Namespace & base_namespace, Names for (const auto & current_namespace : namespaces) { chassert(current_namespace.starts_with(base_namespace)); - getNamespacesRecursive(current_namespace, result); + + if (stop_condition && stop_condition(current_namespace)) + break; + + getNamespacesRecursive(current_namespace, result, stop_condition); } } @@ -175,14 +203,14 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const return namespaces; } -RestCatalog::Tables RestCatalog::getTables(const Namespace & base_namespace) const +RestCatalog::Tables RestCatalog::getTables(const Namespace & base_namespace, size_t limit) const { const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; auto buf = createReadBuffer(endpoint); - return parseTables(*buf, base_namespace); + return parseTables(*buf, base_namespace, limit); } -RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & base_namespace) const +RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const { if (buf.eof()) return {}; @@ -201,9 +229,12 @@ RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::st Tables tables; for (size_t i = 0; i < identifiers_object->size(); ++i) { - auto current_table_json = identifiers_object->get(static_cast(i)).extract(); - auto table_name = current_table_json->get("name").extract(); + const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); + const auto table_name = current_table_json->get("name").extract(); + tables.push_back(base_namespace + "." + table_name); + if (limit && tables.size() >= limit) + break; } return tables; } diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index b33d5accda4..e83aa8eabe9 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -25,6 +25,8 @@ public: bool existsCatalog() const override; + bool empty() const override; + Tables getTables() const override; bool existsTable(const std::string & namespace_name, const std::string & table_name) const override; @@ -48,15 +50,16 @@ private: Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; - void getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result) const; + using StopCondition = std::function; + void getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result, StopCondition stop_condition) const; Namespaces getNamespaces(const Namespace & base_namespace) const; Namespaces parseNamespaces(DB::ReadBuffer & buf, const std::string & base_namespace) const; - Tables getTables(const Namespace & base_namespace) const; + Tables getTables(const Namespace & base_namespace, size_t limit = 0) const; - Tables parseTables(DB::ReadBuffer & buf, const std::string & base_namespace) const; + Tables parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const; bool getTableMetadataImpl( const std::string & namespace_name, diff --git a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml index b529bf7d3ff..860d9e29230 100644 --- a/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml +++ b/tests/integration/compose/docker_compose_iceberg_rest_catalog.yml @@ -22,7 +22,7 @@ services: - AWS_ACCESS_KEY_ID=minio - AWS_SECRET_ACCESS_KEY=minio123 - AWS_REGION=us-east-1 - - CATALOG_WAREHOUSE=s3://warehouse/ + - CATALOG_WAREHOUSE=s3://iceberg_data/ - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO - CATALOG_S3_ENDPOINT=http://minio:9000 minio: @@ -36,8 +36,9 @@ services: default: aliases: - warehouse.minio - expose: - - 9001 + ports: + - 9001:9001 + - 9002:9000 command: ["server", "/data", "--console-address", ":9001"] mc: depends_on: diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 05344b7467c..4341b9cb30a 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -22,7 +22,9 @@ from pyiceberg.types import ( StructType, TimestampType, ) - +import pyarrow as pa +import random +from datetime import datetime, timedelta from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_bucket from helpers.test_tools import TSV, csv_compare @@ -34,9 +36,11 @@ BASE_URL_LOCAL_RAW = "http://localhost:8182" CATALOG_NAME = "demo" DEFAULT_SCHEMA = Schema( - NestedField(field_id=1, name="datetime", field_type=TimestampType(), required=True), - NestedField(field_id=2, name="symbol", field_type=StringType(), required=True), - NestedField(field_id=3, name="bid", field_type=FloatType(), required=False), + NestedField( + field_id=1, name="datetime", field_type=TimestampType(), required=False + ), + NestedField(field_id=2, name="symbol", field_type=StringType(), required=False), + NestedField(field_id=3, name="bid", field_type=DoubleType(), required=False), NestedField(field_id=4, name="ask", field_type=DoubleType(), required=False), NestedField( field_id=5, @@ -52,11 +56,15 @@ DEFAULT_SCHEMA = Schema( required=False, ), ) + +DEFAULT_CREATE_TABLE = "CREATE TABLE {}.`{}.{}`\\n(\\n `datetime` Nullable(DateTime64(6)),\\n `symbol` Nullable(String),\\n `bid` Nullable(Float64),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://minio:9000/warehouse/data\\', \\'minio\\', \\'[HIDDEN]\\')\n" + DEFAULT_PARTITION_SPEC = PartitionSpec( PartitionField( source_id=1, field_id=1000, transform=DayTransform(), name="datetime_day" ) ) + DEFAULT_SORT_ORDER = SortOrder(SortField(source_id=2, transform=IdentityTransform())) @@ -68,13 +76,13 @@ def list_namespaces(): raise Exception(f"Failed to list namespaces: {response.status_code}") -def load_catalog_impl(): +def load_catalog_impl(started_cluster): return load_catalog( CATALOG_NAME, **{ "uri": BASE_URL_LOCAL_RAW, "type": "rest", - "s3.endpoint": f"http://minio:9000", + "s3.endpoint": f"http://localhost:9002", "s3.access-key-id": "minio", "s3.secret-access-key": "minio123", }, @@ -89,25 +97,51 @@ def create_table( partition_spec=DEFAULT_PARTITION_SPEC, sort_order=DEFAULT_SORT_ORDER, ): - catalog.create_table( + return catalog.create_table( identifier=f"{namespace}.{table}", schema=schema, - location=f"s3://warehouse", + location=f"s3://warehouse/data", partition_spec=partition_spec, sort_order=sort_order, ) +def generate_record(): + return { + "datetime": datetime.now(), + "symbol": str("kek"), + "bid": round(random.uniform(100, 200), 2), + "ask": round(random.uniform(200, 300), 2), + "details": {"created_by": "Alice Smith"}, + } + + def create_clickhouse_iceberg_database(started_cluster, node, name): node.query( f""" DROP DATABASE IF EXISTS {name}; CREATE DATABASE {name} ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') -SETTINGS catalog_type = 'rest', storage_endpoint = 'http://{started_cluster.minio_ip}:{started_cluster.minio_port}/' +SETTINGS catalog_type = 'rest', storage_endpoint = 'http://minio:9000/' """ ) +def print_objects(): + minio_client = Minio( + f"localhost:9002", + access_key="minio", + secret_key="minio123", + secure=False, + http_client=urllib3.PoolManager(cert_reqs="CERT_NONE"), + ) + + objects = list(minio_client.list_objects("warehouse", "", recursive=True)) + names = [x.object_name for x in objects] + names.sort() + for name in names: + print(f"Found object: {name}") + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -125,7 +159,6 @@ def started_cluster(): # TODO: properly wait for container time.sleep(10) - #cluster.minio_client.make_bucket("warehouse") yield cluster @@ -133,7 +166,7 @@ def started_cluster(): cluster.shutdown() -def test_simple(started_cluster): +def test_list_tables(started_cluster): node = started_cluster.instances["node1"] root_namespace = "clickhouse" @@ -142,7 +175,7 @@ def test_simple(started_cluster): namespace_1_tables = ["tableA", "tableB"] namespace_2_tables = ["tableC", "tableD"] - catalog = load_catalog_impl() + catalog = load_catalog_impl(started_cluster) for namespace in [namespace_1, namespace_2]: catalog.create_namespace(namespace) @@ -182,13 +215,13 @@ def test_simple(started_cluster): ).strip() ) - expected = f"CREATE TABLE {CATALOG_NAME}.`{namespace_2}.tableC`\\n(\\n `datetime` DateTime64(6),\\n `symbol` String,\\n `bid` Nullable(Float32),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://None:9001/warehouse\\', \\'minio\\', \\'[HIDDEN]\\')\n" + expected = DEFAULT_CREATE_TABLE.format(CATALOG_NAME, namespace_2, "tableC") assert expected == node.query( f"SHOW CREATE TABLE {CATALOG_NAME}.`{namespace_2}.tableC`" ) -def test_different_namespaces(started_cluster): +def test_many_namespaces(started_cluster): node = started_cluster.instances["node1"] namespaces = [ "A", @@ -202,11 +235,9 @@ def test_different_namespaces(started_cluster): "B.CC", ] tables = ["A", "B", "C", "D", "E", "F"] - catalog = load_catalog_impl() + catalog = load_catalog_impl(started_cluster) for namespace in namespaces: - # if namespace in catalog.list_namespaces()["namesoaces"]: - # catalog.drop_namespace(namespace) catalog.create_namespace(namespace) for table in tables: create_table(catalog, namespace, table) @@ -221,3 +252,43 @@ def test_different_namespaces(started_cluster): f"SELECT count() FROM system.tables WHERE database = '{CATALOG_NAME}' and name = '{table_name}'" ) ) + + +def test_select(started_cluster): + node = started_cluster.instances["node1"] + + test_ref = "test_list_tables" + table_name = f"{test_ref}_table" + root_namespace = f"{test_ref}_namespace" + + namespace = f"{root_namespace}.A.B.C" + namespaces_to_create = [ + root_namespace, + f"{root_namespace}.A", + f"{root_namespace}.A.B", + f"{root_namespace}.A.B.C", + ] + + catalog = load_catalog_impl(started_cluster) + + for namespace in namespaces_to_create: + catalog.create_namespace(namespace) + assert len(catalog.list_tables(namespace)) == 0 + + table = create_table(catalog, namespace, table_name) + + num_rows = 10 + data = [generate_record() for _ in range(num_rows)] + df = pa.Table.from_pylist(data) + table.append(df) + + create_clickhouse_iceberg_database(started_cluster, node, CATALOG_NAME) + + expected = DEFAULT_CREATE_TABLE.format(CATALOG_NAME, namespace, table_name) + assert expected == node.query( + f"SHOW CREATE TABLE {CATALOG_NAME}.`{namespace}.{table_name}`" + ) + + assert num_rows == int( + node.query(f"SELECT count() FROM {CATALOG_NAME}.`{namespace}.{table_name}`") + ) From 58edfbe1136bb8c8dd5f0a29b3ecce0d2a3335b9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Nov 2024 20:12:52 +0100 Subject: [PATCH 11/39] Fix style check --- src/Databases/Iceberg/DatabaseIceberg.cpp | 2 +- tests/integration/test_database_iceberg/test.py | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 7efde3a5671..81c39554635 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -136,7 +136,7 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ const auto columns = ColumnsDescription(table_metadata.getSchema()); const auto configuration = getConfiguration(); - /// with_table_structure = false: because there will be no table stucture in table definition AST. + /// with_table_structure = false: because there will be no table structure in table definition AST. StorageObjectStorage::Configuration::initialize(*configuration, args, context_, /* with_table_structure */false); return std::make_shared( diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 4341b9cb30a..37d2bbf0ea8 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -2,9 +2,12 @@ import glob import json import logging import os +import random import time import uuid +from datetime import datetime, timedelta +import pyarrow as pa import pytest import requests import urllib3 @@ -22,9 +25,7 @@ from pyiceberg.types import ( StructType, TimestampType, ) -import pyarrow as pa -import random -from datetime import datetime, timedelta + from helpers.cluster import ClickHouseCluster, ClickHouseInstance, is_arm from helpers.s3_tools import get_file_contents, list_s3_objects, prepare_s3_bucket from helpers.test_tools import TSV, csv_compare From 6c3003c6ce0c41542e410921afcebc6d71b0e51c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 13 Nov 2024 20:30:33 +0100 Subject: [PATCH 12/39] Cleanup --- src/Databases/Iceberg/DatabaseIceberg.cpp | 38 +++++++++------ src/Databases/Iceberg/DatabaseIceberg.h | 10 +++- src/Databases/Iceberg/ICatalog.cpp | 4 +- src/Databases/Iceberg/ICatalog.h | 58 +++++++++++------------ src/Databases/Iceberg/RestCatalog.cpp | 27 ++++------- src/Databases/Iceberg/RestCatalog.h | 8 ++-- 6 files changed, 73 insertions(+), 72 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 81c39554635..311ae9bf2dc 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -101,6 +101,13 @@ std::shared_ptr DatabaseIceberg::getConfigu } } +std::string DatabaseIceberg::getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const +{ + return std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) + / table_metadata.getPath() + / ""; +} + bool DatabaseIceberg::empty() const { return getCatalog(Context::getGlobalContextInstance())->empty(); @@ -115,7 +122,7 @@ bool DatabaseIceberg::isTableExist(const String & name, ContextPtr context_) con StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_) const { auto catalog = getCatalog(context_); - auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation().withSchema(); + auto table_metadata = Iceberg::TableMetadata().withLocation().withSchema(); auto [namespace_name, table_name] = parseTableName(name); if (!catalog->tryGetTableMetadata(namespace_name, table_name, table_metadata)) @@ -126,17 +133,16 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ ASTs args = storage->engine->arguments->children; /// Replace Iceberg Catalog endpoint with storage path endpoint of requested table. - auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) - / table_metadata.getPath() - / ""; + auto table_endpoint = getStorageEndpointForTable(table_metadata); + args[0] = std::make_shared(table_endpoint); - args[0] = std::make_shared(table_endpoint.string()); - - LOG_TEST(log, "Using table endpoint: {}", table_endpoint.string()); + LOG_TEST(log, "Using table endpoint: {}", table_endpoint); const auto columns = ColumnsDescription(table_metadata.getSchema()); const auto configuration = getConfiguration(); - /// with_table_structure = false: because there will be no table structure in table definition AST. + + /// with_table_structure = false: because there will be + /// no table structure in table definition AST. StorageObjectStorage::Configuration::initialize(*configuration, args, context_, /* with_table_structure */false); return std::make_shared( @@ -184,8 +190,9 @@ ASTPtr DatabaseIceberg::getCreateTableQueryImpl( bool /* throw_on_error */) const { auto catalog = getCatalog(context_); - auto table_metadata = Iceberg::ICatalog::TableMetadata().withLocation().withSchema(); - auto [namespace_name, table_name] = parseTableName(name); + auto table_metadata = Iceberg::TableMetadata().withLocation().withSchema(); + + const auto [namespace_name, table_name] = parseTableName(name); catalog->getTableMetadata(namespace_name, table_name, table_metadata); auto create_table_query = std::make_shared(); @@ -203,7 +210,6 @@ ASTPtr DatabaseIceberg::getCreateTableQueryImpl( columns_declare_list->set(columns_declare_list->columns, columns_expression_list); create_table_query->set(create_table_query->columns_list, columns_declare_list); - /// init create query. create_table_query->setTable(name); create_table_query->setDatabase(getDatabaseName()); @@ -217,10 +223,14 @@ ASTPtr DatabaseIceberg::getCreateTableQueryImpl( auto storage_engine_arguments = storage->engine->arguments; if (storage_engine_arguments->children.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected number of arguments: {}", storage_engine_arguments->children.size()); + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unexpected number of arguments: {}", + storage_engine_arguments->children.size()); + } - auto table_endpoint = std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) / table_metadata.getPath(); - storage_engine_arguments->children[0] = std::make_shared(table_endpoint.string()); + auto table_endpoint = getStorageEndpointForTable(table_metadata); + storage_engine_arguments->children[0] = std::make_shared(table_endpoint); return create_table_query; } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 41ed17f58d0..5a87e3179af 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -10,6 +10,10 @@ namespace DB { +/// TODO: +/// - http basic auth for catalog +/// - tests with azure, hdfs, local + class DatabaseIceberg final : public IDatabase, WithContext { public: @@ -20,7 +24,6 @@ public: ASTPtr database_engine_definition_); String getEngineName() const override { return "Iceberg"; } - String getMetadataPath() const override { return ""; } bool canContainMergeTreeTables() const override { return false; } bool canContainDistributedTables() const override { return false; } @@ -44,13 +47,18 @@ protected: ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; private: + /// Iceberg Catalog url. const std::string url; + /// SETTINGS from CREATE query. const DatabaseIcebergSettings settings; + /// Database engine definition taken from initial CREATE DATABASE query. const ASTPtr database_engine_definition; + const LoggerPtr log; std::unique_ptr getCatalog(ContextPtr context_) const; std::shared_ptr getConfiguration() const; + std::string getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const; }; } diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index a67d88fc555..d07d9613e1a 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -10,7 +10,7 @@ namespace DB::ErrorCodes namespace Iceberg { -std::string ICatalog::TableMetadata::getPath() const +std::string TableMetadata::getPath() const { if (!with_location) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); @@ -21,7 +21,7 @@ std::string ICatalog::TableMetadata::getPath() const throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); } -const DB::NamesAndTypesList & ICatalog::TableMetadata::getSchema() const +const DB::NamesAndTypesList & TableMetadata::getSchema() const { if (!with_schema) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index a9e8c24b01b..7bd2cf3010c 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -5,22 +5,41 @@ namespace Iceberg { +class TableMetadata +{ +friend class RestCatalog; + +public: + TableMetadata() = default; + + std::string getPath() const; + + const DB::NamesAndTypesList & getSchema() const; + + TableMetadata & withLocation() { with_location = true; return *this; } + TableMetadata & withSchema() { with_schema = true; return *this; } + +private: + /// starts with s3://, file://, etc + std::string location; + /// column names and types + DB::NamesAndTypesList schema; + + bool with_location = false; + bool with_schema = false; +}; + + class ICatalog { public: - using Namespace = std::string; - using Namespaces = std::vector; - using Table = std::string; - using Tables = std::vector
; - - class TableMetadata; + using Namespaces = std::vector; + using Tables = std::vector; explicit ICatalog(const std::string & catalog_name_) : catalog_name(catalog_name_) {} virtual ~ICatalog() = default; - virtual bool existsCatalog() const = 0; - virtual bool empty() const = 0; virtual Tables getTables() const = 0; @@ -43,27 +62,4 @@ protected: const std::string catalog_name; }; -class ICatalog::TableMetadata -{ -friend class RestCatalog; - -public: - TableMetadata() = default; - - std::string getPath() const; - - const DB::NamesAndTypesList & getSchema() const; - - TableMetadata & withLocation() { with_location = true; return *this; } - TableMetadata & withSchema() { with_schema = true; return *this; } - -private: - /// starts with s3://, file://, etc - std::string location; - /// column names and types - DB::NamesAndTypesList schema; - - bool with_location = false; - bool with_schema = false; -}; } diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 86abec72c29..729ed641494 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -38,20 +38,6 @@ RestCatalog::RestCatalog( { } -bool RestCatalog::existsCatalog() const -{ - try - { - createReadBuffer(namespaces_endpoint)->eof(); - return true; - } - catch (...) - { - DB::tryLogCurrentException(log); - return false; - } -} - bool RestCatalog::empty() const { try @@ -108,7 +94,7 @@ RestCatalog::Tables RestCatalog::getTables() const return tables; } -void RestCatalog::getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result, StopCondition stop_condition) const +void RestCatalog::getNamespacesRecursive(const std::string & base_namespace, Namespaces & result, StopCondition stop_condition) const { auto namespaces = getNamespaces(base_namespace); result.reserve(result.size() + namespaces.size()); @@ -194,16 +180,19 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const if (current_namespace_array->size() == 0) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expected namespace array to be non-empty"); - const int current_namespace_idx = static_cast(current_namespace_array->size()) - 1; - const auto current_namespace = current_namespace_array->get(current_namespace_idx).extract(); - const auto full_namespace = base_namespace.empty() ? current_namespace : base_namespace + "." + current_namespace; + const int idx = static_cast(current_namespace_array->size()) - 1; + const auto current_namespace = current_namespace_array->get(idx).extract(); + const auto full_namespace = base_namespace.empty() + ? current_namespace + : base_namespace + "." + current_namespace; + namespaces.push_back(full_namespace); } return namespaces; } -RestCatalog::Tables RestCatalog::getTables(const Namespace & base_namespace, size_t limit) const +RestCatalog::Tables RestCatalog::getTables(const std::string & base_namespace, size_t limit) const { const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; auto buf = createReadBuffer(endpoint); diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index e83aa8eabe9..12ff31ad8d1 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -23,8 +23,6 @@ public: ~RestCatalog() override = default; - bool existsCatalog() const override; - bool empty() const override; Tables getTables() const override; @@ -51,13 +49,13 @@ private: Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; using StopCondition = std::function; - void getNamespacesRecursive(const Namespace & base_namespace, Namespaces & result, StopCondition stop_condition) const; + void getNamespacesRecursive(const std::string & base_namespace, Namespaces & result, StopCondition stop_condition) const; - Namespaces getNamespaces(const Namespace & base_namespace) const; + Namespaces getNamespaces(const std::string & base_namespace) const; Namespaces parseNamespaces(DB::ReadBuffer & buf, const std::string & base_namespace) const; - Tables getTables(const Namespace & base_namespace, size_t limit = 0) const; + Tables getTables(const std::string & base_namespace, size_t limit = 0) const; Tables parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const; From 326c91c02b7157c5f2834ff03f77a5288af60956 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Nov 2024 13:56:09 +0100 Subject: [PATCH 13/39] Fix test --- src/Databases/Iceberg/DatabaseIceberg.h | 2 +- .../integration/test_database_iceberg/test.py | 49 ++++++++++++------- 2 files changed, 32 insertions(+), 19 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 5a87e3179af..f7b7544f80a 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -11,7 +11,7 @@ namespace DB { /// TODO: -/// - http basic auth for catalog +/// - auth: oauth, bearer token? /// - tests with azure, hdfs, local class DatabaseIceberg final : public IDatabase, WithContext diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 37d2bbf0ea8..310880c282a 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -58,7 +58,7 @@ DEFAULT_SCHEMA = Schema( ), ) -DEFAULT_CREATE_TABLE = "CREATE TABLE {}.`{}.{}`\\n(\\n `datetime` Nullable(DateTime64(6)),\\n `symbol` Nullable(String),\\n `bid` Nullable(Float64),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://minio:9000/warehouse/data\\', \\'minio\\', \\'[HIDDEN]\\')\n" +DEFAULT_CREATE_TABLE = "CREATE TABLE {}.`{}.{}`\\n(\\n `datetime` Nullable(DateTime64(6)),\\n `symbol` Nullable(String),\\n `bid` Nullable(Float64),\\n `ask` Nullable(Float64),\\n `details` Tuple(created_by Nullable(String))\\n)\\nENGINE = Iceberg(\\'http://minio:9000/warehouse/data/\\', \\'minio\\', \\'[HIDDEN]\\')\n" DEFAULT_PARTITION_SPEC = PartitionSpec( PartitionField( @@ -170,9 +170,9 @@ def started_cluster(): def test_list_tables(started_cluster): node = started_cluster.instances["node1"] - root_namespace = "clickhouse" - namespace_1 = "clickhouse.testA.A" - namespace_2 = "clickhouse.testB.B" + root_namespace = f"clickhouse_{uuid.uuid4()}" + namespace_1 = f"{root_namespace}.testA.A" + namespace_2 = f"{root_namespace}.testB.B" namespace_1_tables = ["tableA", "tableB"] namespace_2_tables = ["tableC", "tableD"] @@ -181,8 +181,19 @@ def test_list_tables(started_cluster): for namespace in [namespace_1, namespace_2]: catalog.create_namespace(namespace) - assert root_namespace in list_namespaces()["namespaces"][0][0] - assert [(root_namespace,)] == catalog.list_namespaces() + found = False + for namespace_list in list_namespaces()["namespaces"]: + if root_namespace == namespace_list[0]: + found = True + break + assert found + + found = False + for namespace_list in catalog.list_namespaces(): + if root_namespace == namespace_list[0]: + found = True + break + assert found for namespace in [namespace_1, namespace_2]: assert len(catalog.list_tables(namespace)) == 0 @@ -205,14 +216,14 @@ def test_list_tables(started_cluster): assert ( tables_list == node.query( - f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' ORDER BY name" + f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' and name ILIKE '{root_namespace}%' ORDER BY name" ).strip() ) node.restart_clickhouse() assert ( tables_list == node.query( - f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' ORDER BY name" + f"SELECT name FROM system.tables WHERE database = '{CATALOG_NAME}' and name ILIKE '{root_namespace}%' ORDER BY name" ).strip() ) @@ -224,16 +235,18 @@ def test_list_tables(started_cluster): def test_many_namespaces(started_cluster): node = started_cluster.instances["node1"] + root_namespace_1 = f"A_{uuid.uuid4()}" + root_namespace_2 = f"B_{uuid.uuid4()}" namespaces = [ - "A", - "A.B.C", - "A.B.C.D", - "A.B.C.D.E", - "A.B.C.D.E.F", - "A.B.C.D.E.FF", - "B", - "B.C", - "B.CC", + f"{root_namespace_1}", + f"{root_namespace_1}.B.C", + f"{root_namespace_1}.B.C.D", + f"{root_namespace_1}.B.C.D.E", + f"{root_namespace_1}.B.C.D.E.F", + f"{root_namespace_1}.B.C.D.E.FF", + f"{root_namespace_2}", + f"{root_namespace_2}.C", + f"{root_namespace_2}.CC", ] tables = ["A", "B", "C", "D", "E", "F"] catalog = load_catalog_impl(started_cluster) @@ -258,7 +271,7 @@ def test_many_namespaces(started_cluster): def test_select(started_cluster): node = started_cluster.instances["node1"] - test_ref = "test_list_tables" + test_ref = f"test_list_tables_{uuid.uuid4()}" table_name = f"{test_ref}_table" root_namespace = f"{test_ref}_namespace" From 0e14b49298932bae829aa5d1ff848b01e9e2e5af Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 14 Nov 2024 19:56:51 +0100 Subject: [PATCH 14/39] Support auth header --- src/Databases/Iceberg/DatabaseIceberg.cpp | 28 +++++++++++++++++++ src/Databases/Iceberg/DatabaseIceberg.h | 5 +++- .../Iceberg/DatabaseIcebergSettings.cpp | 1 + 3 files changed, 33 insertions(+), 1 deletion(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 311ae9bf2dc..39ed6118af1 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -1,6 +1,8 @@ #include #if USE_AVRO +#include + #include #include #include @@ -27,6 +29,7 @@ namespace DB namespace DatabaseIcebergSetting { extern const DatabaseIcebergSettingsString storage_endpoint; + extern const DatabaseIcebergSettingsString auth_header; extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; extern const DatabaseIcebergSettingsDatabaseIcebergStorageType storage_type; } @@ -52,6 +55,20 @@ namespace auto namespace_name = name.substr(0, name.size() - table_name.size() - 1); return {namespace_name, table_name}; } + + void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) + { + const auto & user_info = request_uri.getUserInfo(); + if (!user_info.empty()) + { + std::size_t n = user_info.find(':'); + if (n != std::string::npos) + { + credentials.setUsername(user_info.substr(0, n)); + credentials.setPassword(user_info.substr(n + 1)); + } + } + } } DatabaseIceberg::DatabaseIceberg( @@ -65,6 +82,17 @@ DatabaseIceberg::DatabaseIceberg( , database_engine_definition(database_engine_definition_) , log(getLogger("DatabaseIceberg(" + database_name_ + ")")) { + setCredentials(credentials, Poco::URI(url)); + + const auto auth_header = settings[DatabaseIcebergSetting::auth_header].value; + if (!auth_header.empty()) + { + auto pos = auth_header.find(':'); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); + headers.emplace_back(auth_header.substr(0, pos), auth_header.substr(pos + 1)); + } + } std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr context_) const diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index f7b7544f80a..230920b1190 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -53,8 +54,10 @@ private: const DatabaseIcebergSettings settings; /// Database engine definition taken from initial CREATE DATABASE query. const ASTPtr database_engine_definition; - const LoggerPtr log; + /// Crendetials to authenticate Iceberg Catalog. + Poco::Net::HTTPBasicCredentials credentials; + HTTPHeaderEntries headers; std::unique_ptr getCatalog(ContextPtr context_) const; std::shared_ptr getConfiguration() const; diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index 8383c372a52..5017686b0d7 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes #define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE, ALIAS) \ DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ + DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ #define LIST_OF_DATABASE_ICEBERG_SETTINGS(M, ALIAS) \ From 32ff7d2722c8ec70e7b692d4980d09760ddbea93 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 15 Nov 2024 11:41:54 +0100 Subject: [PATCH 15/39] Fix build and test --- src/Databases/Iceberg/DatabaseIceberg.cpp | 12 ++++++++++++ tests/integration/test_database_iceberg/test.py | 4 +--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 39ed6118af1..8f46baf0405 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -110,22 +110,34 @@ std::shared_ptr DatabaseIceberg::getConfigu { switch (settings[DatabaseIcebergSetting::storage_type].value) { +#if USE_AWS_S3 case DB::DatabaseIcebergStorageType::S3: { return std::make_shared(); } +#endif +#if USE_AZURE_BLOB_STORAGE case DB::DatabaseIcebergStorageType::Azure: { return std::make_shared(); } +#endif +#if USE_HDFS case DB::DatabaseIcebergStorageType::HDFS: { return std::make_shared(); } +#endif case DB::DatabaseIcebergStorageType::Local: { return std::make_shared(); } +#if !USE_AWS_S3 || !USE_AZURE_BLOB_STORAGE || !USE_HDFS + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Server does not contain support for storage type {}", + settings[DatabaseIcebergSetting::storage_type].value); +#endif } } diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 310880c282a..51d89444c2e 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -242,13 +242,11 @@ def test_many_namespaces(started_cluster): f"{root_namespace_1}.B.C", f"{root_namespace_1}.B.C.D", f"{root_namespace_1}.B.C.D.E", - f"{root_namespace_1}.B.C.D.E.F", - f"{root_namespace_1}.B.C.D.E.FF", f"{root_namespace_2}", f"{root_namespace_2}.C", f"{root_namespace_2}.CC", ] - tables = ["A", "B", "C", "D", "E", "F"] + tables = ["A", "B", "C"] catalog = load_catalog_impl(started_cluster) for namespace in namespaces: From 4e2549bcf3b60063172796ef6df06f761dc63c18 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Nov 2024 13:11:16 +0100 Subject: [PATCH 16/39] Fixes & improvements --- src/Databases/Iceberg/DatabaseIceberg.cpp | 75 ++++-- src/Databases/Iceberg/DatabaseIceberg.h | 4 +- .../Iceberg/DatabaseIcebergSettings.cpp | 2 + src/Databases/Iceberg/ICatalog.cpp | 65 +++++- src/Databases/Iceberg/ICatalog.h | 31 ++- src/Databases/Iceberg/RestCatalog.cpp | 216 +++++++++++++++--- src/Databases/Iceberg/RestCatalog.h | 34 ++- 7 files changed, 347 insertions(+), 80 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 8f46baf0405..c22d4747f83 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -28,10 +28,12 @@ namespace DB { namespace DatabaseIcebergSetting { - extern const DatabaseIcebergSettingsString storage_endpoint; - extern const DatabaseIcebergSettingsString auth_header; extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; extern const DatabaseIcebergSettingsDatabaseIcebergStorageType storage_type; + extern const DatabaseIcebergSettingsString warehouse; + extern const DatabaseIcebergSettingsString catalog_credential; + extern const DatabaseIcebergSettingsString auth_header; + extern const DatabaseIcebergSettingsString storage_endpoint; } namespace ErrorCodes @@ -55,44 +57,55 @@ namespace auto namespace_name = name.substr(0, name.size() - table_name.size() - 1); return {namespace_name, table_name}; } - - void setCredentials(Poco::Net::HTTPBasicCredentials & credentials, const Poco::URI & request_uri) - { - const auto & user_info = request_uri.getUserInfo(); - if (!user_info.empty()) - { - std::size_t n = user_info.find(':'); - if (n != std::string::npos) - { - credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n + 1)); - } - } - } } DatabaseIceberg::DatabaseIceberg( const std::string & database_name_, const std::string & url_, const DatabaseIcebergSettings & settings_, - ASTPtr database_engine_definition_) + ASTPtr database_engine_definition_, + ContextPtr context_) : IDatabase(database_name_) , url(url_) , settings(settings_) , database_engine_definition(database_engine_definition_) , log(getLogger("DatabaseIceberg(" + database_name_ + ")")) { - setCredentials(credentials, Poco::URI(url)); - const auto auth_header = settings[DatabaseIcebergSetting::auth_header].value; + LOG_TEST(log, "Auth header: {}", auth_header); if (!auth_header.empty()) { auto pos = auth_header.find(':'); if (pos == std::string::npos) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); headers.emplace_back(auth_header.substr(0, pos), auth_header.substr(pos + 1)); + + LOG_TEST(log, "Added header: {}={}", headers.back().name, headers.back().value); } + validateSettings(context_); +} + +void DatabaseIceberg::validateSettings(const ContextPtr & context_) +{ + if (settings[DatabaseIcebergSetting::warehouse].value.empty()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "`warehouse` setting cannot be empty. " + "Please specify 'SETTINGS warehouse=' in the CREATE DATABASE query"); + } + + if (!settings[DatabaseIcebergSetting::storage_type].changed) + { + auto catalog = getCatalog(context_); + const auto storage_type = catalog->getStorageType(); + if (!storage_type) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Storage type is not found in catalog config. " + "Please specify it manually via 'SETTINGS storage_type=' in CREATE DATABASE query"); + } + } } std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr context_) const @@ -101,7 +114,12 @@ std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr contex { case DB::DatabaseIcebergCatalogType::REST: { - return std::make_unique(getDatabaseName(), url, context_); + return std::make_unique( + settings[DatabaseIcebergSetting::warehouse].value, + url, + settings[DatabaseIcebergSetting::catalog_credential].value, + headers, + context_); } } } @@ -143,9 +161,17 @@ std::shared_ptr DatabaseIceberg::getConfigu std::string DatabaseIceberg::getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const { - return std::filesystem::path(settings[DatabaseIcebergSetting::storage_endpoint].value) - / table_metadata.getPath() - / ""; + auto endpoint_from_settings = settings[DatabaseIcebergSetting::storage_endpoint].value; + if (!endpoint_from_settings.empty()) + { + return std::filesystem::path(endpoint_from_settings) + / table_metadata.getLocation(/* path_only */true) + / ""; + } + else + { + return std::filesystem::path(table_metadata.getLocation(/* path_only */false)) / ""; + } } bool DatabaseIceberg::empty() const @@ -307,7 +333,8 @@ void registerDatabaseIceberg(DatabaseFactory & factory) args.database_name, url, database_settings, - database_engine_define->clone()); + database_engine_define->clone(), + args.context); }; factory.registerDatabase("Iceberg", create_fn, { .supports_arguments = true, .supports_settings = true }); } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 230920b1190..c9147bcd7ab 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -22,7 +22,8 @@ public: const std::string & database_name_, const std::string & url_, const DatabaseIcebergSettings & settings_, - ASTPtr database_engine_definition_); + ASTPtr database_engine_definition_, + ContextPtr context_); String getEngineName() const override { return "Iceberg"; } @@ -59,6 +60,7 @@ private: Poco::Net::HTTPBasicCredentials credentials; HTTPHeaderEntries headers; + void validateSettings(const ContextPtr & context_); std::unique_ptr getCatalog(ContextPtr context_) const; std::shared_ptr getConfiguration() const; std::string getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const; diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index 5017686b0d7..1dc1ba1f61c 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -17,6 +17,8 @@ namespace ErrorCodes #define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE, ALIAS) \ DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ + DECLARE(String, catalog_credential, "", "", 0) \ + DECLARE(String, warehouse, "", "Warehouse name inside the catalog", 0) \ DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index d07d9613e1a..6b6ecc85daf 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include namespace DB::ErrorCodes { @@ -10,22 +12,75 @@ namespace DB::ErrorCodes namespace Iceberg { -std::string TableMetadata::getPath() const +void TableMetadata::setLocation(const std::string & location_) { if (!with_location) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); - if (location.starts_with("s3://")) - return location.substr(std::strlen("s3://")); + /// Location has format: + /// s3:///path/to/table/data. + /// We want to split s3:// and path/to/table/data. + + auto pos = location_.find("://"); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected location format: {}", location_); + + auto pos_to_bucket = pos + std::strlen("://"); + auto pos_to_path = location_.substr(pos_to_bucket).find('/'); + + if (pos_to_path == std::string::npos) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected location format: {}", location_); + + pos_to_path = pos_to_bucket + pos_to_path; + + location_without_path = location_.substr(0, pos_to_path); + path = location_.substr(pos_to_path + 1); + + LOG_TEST(getLogger("TableMetadata"), + "Parsed location without path: {}, path: {}", + location_without_path, path); +} + +std::string TableMetadata::getLocation(bool path_only) const +{ + if (!with_location) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); + + if (path_only) + return path; else - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); + return std::filesystem::path(location_without_path) / path; +} + +void TableMetadata::setSchema(const DB::NamesAndTypesList & schema_) +{ + if (!with_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + schema = schema_; } const DB::NamesAndTypesList & TableMetadata::getSchema() const { if (!with_schema) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + return schema; } +StorageType ICatalog::getStorageType(const std::string & location) +{ + auto pos = location.find("://"); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); + + auto storage_type_str = location.substr(0, pos); + auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); + + if (!storage_type) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_str); + + return *storage_type; +} + } diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index 7bd2cf3010c..4c61c87998d 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -1,27 +1,36 @@ #pragma once #include #include +#include namespace Iceberg { +using StorageType = DB::DatabaseIcebergStorageType; class TableMetadata { -friend class RestCatalog; - public: TableMetadata() = default; - std::string getPath() const; - - const DB::NamesAndTypesList & getSchema() const; - TableMetadata & withLocation() { with_location = true; return *this; } TableMetadata & withSchema() { with_schema = true; return *this; } + std::string getLocation(bool path_only) const; + std::string getLocation() const; + std::string getLocationWithoutPath() const; + + const DB::NamesAndTypesList & getSchema() const; + + bool requiresLocation() const { return with_location; } + bool requiresSchema() const { return with_schema; } + + void setLocation(const std::string & location_); + void setSchema(const DB::NamesAndTypesList & schema_); + private: /// starts with s3://, file://, etc - std::string location; + std::string location_without_path; + std::string path; /// column names and types DB::NamesAndTypesList schema; @@ -36,7 +45,7 @@ public: using Namespaces = std::vector; using Tables = std::vector; - explicit ICatalog(const std::string & catalog_name_) : catalog_name(catalog_name_) {} + explicit ICatalog(const std::string & warehouse_) : warehouse(warehouse_) {} virtual ~ICatalog() = default; @@ -58,8 +67,12 @@ public: const std::string & table_name, TableMetadata & result) const = 0; + virtual std::optional getStorageType() const = 0; + protected: - const std::string catalog_name; + const std::string warehouse; + + static StorageType getStorageType(const std::string & location); }; } diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 729ed641494..cadb929ed5d 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -13,7 +13,6 @@ #include #include -#include #include namespace DB::ErrorCodes @@ -25,17 +24,178 @@ namespace DB::ErrorCodes namespace Iceberg { +static constexpr auto config_endpoint = "config"; static constexpr auto namespaces_endpoint = "namespaces"; -RestCatalog::RestCatalog( - const std::string & catalog_name_, - const std::string & base_url_, - DB::ContextPtr context_) - : ICatalog(catalog_name_) - , DB::WithContext(context_) - , base_url(base_url_) - , log(getLogger("RestCatalog(" + catalog_name_ + ")")) +std::string RestCatalog::Config::toString() const { + DB::WriteBufferFromOwnString wb; + + if (!prefix.empty()) + wb << "prefix: " << prefix.string() << ", "; + if (!default_base_location.empty()) + wb << "default_base_location: " << default_base_location << ", "; + + return wb.str(); +} + +RestCatalog::RestCatalog( + const std::string & warehouse_, + const std::string & base_url_, + const std::string & catalog_credential_, + const DB::HTTPHeaderEntries & headers_, + DB::ContextPtr context_) + : ICatalog(warehouse_) + , DB::WithContext(context_) + , log(getLogger("RestCatalog(" + warehouse_ + ")")) + , base_url(base_url_) + , headers(headers_) +{ + if (!catalog_credential_.empty()) + { + auto pos = catalog_credential_.find(':'); + if (pos == std::string::npos) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Unexpected format of catalog credential: " + "expected client_id and client_secret separated by `:`"); + } + client_id = catalog_credential_.substr(0, pos); + client_secret = catalog_credential_.substr(pos + 1); + + /// TODO: remove before merge. + LOG_TEST(log, "Client id: {}, client secret: {}", client_id, client_secret); + } + LOG_TEST(log, "kssenii 1"); + config = loadConfig(); +} + +RestCatalog::Config RestCatalog::loadConfig() +{ + LOG_TEST(log, "kssenii 2"); + if (!client_id.empty()) + { + static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; + + const auto & context = getContext(); + const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + // Poco::URI::QueryParameters params = { + // {"grant_type", "client_credentials"}, + // {"scope", "PRINCIPAL_ROLE:ALL"}, + // {"client_id", client_id}, + // {"client_secret", client_secret}}; + + Poco::JSON::Object json; + json.set("grant_type", "client_credentials"); + json.set("scope", "PRINCIPAL_ROLE:ALL"); + json.set("client_id", client_id); + json.set("client_secret", client_secret); + + LOG_TEST(log, "kssenii 3"); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + std::string json_str = oss.str(); + + LOG_TEST(log, "kssenii 4"); + Poco::URI url(base_url / oauth_tokens_endpoint); + // url.setQueryParameters(params); + + LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); + + auto callback = [&](std::ostream & out) + { + out << json_str; + }; + + auto wb = DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withOutCallback(callback) + .withSkipNotFound(false) + .withHeaders(headers) + .create(credentials); + + json_str.clear(); + readJSONObjectPossiblyInvalid(json_str, *wb); + + LOG_TEST(log, "Received token result: {}", json_str); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var res_json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = res_json.extract(); + + auto access_token = object->get("access_token").extract(); + headers.emplace_back("Authorization", "Bearer " + access_token); + } + + Poco::URI::QueryParameters params = {{"warehouse", warehouse}}; + auto buf = createReadBuffer(config_endpoint, params); + + std::string json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + + LOG_TEST(log, "Received config result: {}", json_str); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); + + Config result; + + auto defaults_object = object->get("defaults").extract(); + parseConfig(defaults_object, result); + + auto overrides_object = object->get("overrides").extract(); + parseConfig(overrides_object, result); + + LOG_TEST(log, "Parsed config: {}", result.toString()); + return result; +} + +void RestCatalog::parseConfig(const Poco::JSON::Object::Ptr & object, Config & result) +{ + if (!object) + return; + + if (object->has("prefix")) + result.prefix = object->get("prefix").extract(); + + if (object->has("default-base-location")) + result.default_base_location = object->get("default-base-location").extract(); +} + +std::optional RestCatalog::getStorageType() const +{ + if (config.default_base_location.empty()) + return std::nullopt; + return ICatalog::getStorageType(config.default_base_location); +} + +DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( + const std::string & endpoint, + const Poco::URI::QueryParameters & params) const +{ + const auto & context = getContext(); + const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + + Poco::URI url(base_url / endpoint); + if (!params.empty()) + url.setQueryParameters(params); + + LOG_TEST(log, "Requesting: {}", url.toString()); + + return DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSkipNotFound(false) + .withHeaders(headers) + .create(credentials); } bool RestCatalog::empty() const @@ -62,24 +222,6 @@ bool RestCatalog::empty() const } } -DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer(const std::string & endpoint, const Poco::URI::QueryParameters & params) const -{ - const auto & context = getContext(); - const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); - - Poco::URI url(base_url / endpoint); - if (!params.empty()) - url.setQueryParameters(params); - - return DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) - .withHostFilter(&getContext()->getRemoteHostFilter()) - .withSkipNotFound(true) - .create(credentials); -} - RestCatalog::Tables RestCatalog::getTables() const { Namespaces namespaces; @@ -133,7 +275,7 @@ RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_name try { - auto buf = createReadBuffer(namespaces_endpoint, params); + auto buf = createReadBuffer(config.prefix / namespaces_endpoint, params); auto namespaces = parseNamespaces(*buf, base_namespace); LOG_TEST(log, "Loaded {} namespaces", namespaces.size()); return namespaces; @@ -142,7 +284,7 @@ RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_name { std::string message = fmt::format( "Received error while fetching list of namespaces from iceberg catalog `{}`. ", - catalog_name); + warehouse); if (e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_NOT_FOUND) message += "Namespace provided in the `parent` query parameter is not found. "; @@ -195,7 +337,7 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const RestCatalog::Tables RestCatalog::getTables(const std::string & base_namespace, size_t limit) const { const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; - auto buf = createReadBuffer(endpoint); + auto buf = createReadBuffer(config.prefix / endpoint); return parseTables(*buf, base_namespace, limit); } @@ -267,7 +409,7 @@ bool RestCatalog::getTableMetadataImpl( LOG_TEST(log, "Checking table {} in namespace {}", table_name, namespace_name); const auto endpoint = std::string(namespaces_endpoint) + "/" + namespace_name + "/tables/" + table_name; - auto buf = createReadBuffer(endpoint); + auto buf = createReadBuffer(config.prefix / endpoint); if (buf->eof()) { @@ -288,17 +430,17 @@ bool RestCatalog::getTableMetadataImpl( if (!metadata_object) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); - if (result.with_location) + if (result.requiresLocation()) { - result.location = metadata_object->get("location").extract(); - - LOG_TEST(log, "Location for table {}: {}", table_name, result.location); + const auto location = metadata_object->get("location").extract(); + result.setLocation(location); + LOG_TEST(log, "Location for table {}: {}", table_name, location); } - if (result.with_schema) + if (result.requiresSchema()) { int format_version = metadata_object->getValue("format-version"); - result.schema = DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first; + result.setSchema(DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first); } return true; } diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 12ff31ad8d1..5415ff73a7e 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -2,8 +2,10 @@ #include #include #include +#include #include #include +#include namespace DB { @@ -17,8 +19,10 @@ class RestCatalog final : public ICatalog, private DB::WithContext { public: explicit RestCatalog( - const std::string & catalog_name_, + const std::string & warehouse_, const std::string & base_url_, + const std::string & catalog_credential_, + const DB::HTTPHeaderEntries & headers_, DB::ContextPtr context_); ~RestCatalog() override = default; @@ -39,12 +43,31 @@ public: const std::string & table_name, TableMetadata & result) const override; + std::optional getStorageType() const override; + private: - const std::filesystem::path base_url; - Poco::Net::HTTPBasicCredentials credentials{}; LoggerPtr log; - DB::ReadWriteBufferFromHTTPPtr createReadBuffer(const std::string & endpoint, const Poco::URI::QueryParameters & params = {}) const; + struct Config + { + std::filesystem::path prefix; + std::string default_base_location; + + std::string toString() const; + }; + + const std::filesystem::path base_url; + DB::HTTPHeaderEntries headers; + std::string client_id; + std::string client_secret; + Config config; + + Poco::Net::HTTPBasicCredentials credentials{}; + + + DB::ReadWriteBufferFromHTTPPtr createReadBuffer( + const std::string & endpoint, + const Poco::URI::QueryParameters & params = {}) const; Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; @@ -63,6 +86,9 @@ private: const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const; + + Config loadConfig(); + static void parseConfig(const Poco::JSON::Object::Ptr & object, Config & result); }; } From b34dfca0c56f63fb8caa655a28ffe6b860379936 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Nov 2024 14:52:10 +0100 Subject: [PATCH 17/39] Speedup tables loading --- src/Databases/Iceberg/DatabaseIceberg.cpp | 47 +++++++++-- src/Databases/Iceberg/DatabaseIceberg.h | 4 +- src/Databases/Iceberg/RestCatalog.cpp | 83 +++++++++++++++---- src/Databases/Iceberg/RestCatalog.h | 8 +- .../ObjectStorage/StorageObjectStorage.cpp | 6 +- .../ObjectStorage/StorageObjectStorage.h | 3 +- 6 files changed, 121 insertions(+), 30 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index c22d4747f83..d701e73d429 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -23,6 +23,13 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric MergeTreeDataSelectExecutorThreads; + extern const Metric MergeTreeDataSelectExecutorThreadsActive; + extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; +} + namespace DB { @@ -108,20 +115,24 @@ void DatabaseIceberg::validateSettings(const ContextPtr & context_) } } -std::unique_ptr DatabaseIceberg::getCatalog(ContextPtr context_) const +std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const { + if (catalog_impl) + return catalog_impl; + switch (settings[DatabaseIcebergSetting::catalog_type].value) { case DB::DatabaseIcebergCatalogType::REST: { - return std::make_unique( + catalog_impl = std::make_shared( settings[DatabaseIcebergSetting::warehouse].value, url, settings[DatabaseIcebergSetting::catalog_credential].value, headers, - context_); + Context::getGlobalContextInstance()); } } + return catalog_impl; } std::shared_ptr DatabaseIceberg::getConfiguration() const @@ -220,7 +231,10 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ /* constraints */ConstraintsDescription{}, /* comment */"", getFormatSettings(context_), - LoadingStrictnessLevel::CREATE); + LoadingStrictnessLevel::CREATE, + /* distributed_processing */false, + /* partition_by */nullptr, + /* lazy_init */true); } DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( @@ -230,15 +244,34 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( { Tables tables; auto catalog = getCatalog(context_); - for (const auto & table_name : catalog->getTables()) + + auto iceberg_tables = catalog->getTables(); + size_t num_threads = std::min(10, iceberg_tables.size()); + ThreadPool pool( + CurrentMetrics::MergeTreeDataSelectExecutorThreads, + CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, + CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled, + num_threads); + + DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); + std::mutex mutexx; + + for (const auto & table_name : iceberg_tables) { if (filter_by_table_name && !filter_by_table_name(table_name)) continue; - auto storage = tryGetTable(table_name, context_); - tables.emplace(table_name, storage); + runner([&]{ + auto storage = tryGetTable(table_name, context_); + { + std::lock_guard lock(mutexx); + [[maybe_unused]] bool inserted = tables.emplace(table_name, storage).second; + chassert(inserted); + } + }); } + runner.waitForAllToFinishAndRethrowFirstError(); return std::make_unique(tables, getDatabaseName()); } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index c9147bcd7ab..a355e316c1a 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -60,8 +60,10 @@ private: Poco::Net::HTTPBasicCredentials credentials; HTTPHeaderEntries headers; + mutable std::shared_ptr catalog_impl; + void validateSettings(const ContextPtr & context_); - std::unique_ptr getCatalog(ContextPtr context_) const; + std::shared_ptr getCatalog(ContextPtr context_) const; std::shared_ptr getConfiguration() const; std::string getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const; }; diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index cadb929ed5d..76c5e897af9 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -9,6 +10,7 @@ #include #include +#include #include #include @@ -21,6 +23,13 @@ namespace DB::ErrorCodes extern const int LOGICAL_ERROR; } +namespace CurrentMetrics +{ + extern const Metric MergeTreeDataSelectExecutorThreads; + extern const Metric MergeTreeDataSelectExecutorThreadsActive; + extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; +} + namespace Iceberg { @@ -79,11 +88,6 @@ RestCatalog::Config RestCatalog::loadConfig() const auto & context = getContext(); const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); - // Poco::URI::QueryParameters params = { - // {"grant_type", "client_credentials"}, - // {"scope", "PRINCIPAL_ROLE:ALL"}, - // {"client_id", client_id}, - // {"client_secret", client_secret}}; Poco::JSON::Object json; json.set("grant_type", "client_credentials"); @@ -91,16 +95,33 @@ RestCatalog::Config RestCatalog::loadConfig() json.set("client_id", client_id); json.set("client_secret", client_secret); - LOG_TEST(log, "kssenii 3"); - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); Poco::JSON::Stringifier::stringify(json, oss); std::string json_str = oss.str(); - LOG_TEST(log, "kssenii 4"); + Poco::URI::QueryParameters params = { + {"Content-Type", "application/x-www-form-urlencoded"}, + {"Content-Length", DB::toString(json_str.size())} + }; + // {"grant_type", "client_credentials"}, + // {"scope", "PRINCIPAL_ROLE:ALL"}, + // {"client_id", client_id}, + // {"client_secret", client_secret}}; + + + // LOG_TEST(log, "kssenii 3"); + + // DB::HTMLForm form(context->getSettingsRef()); + // form.add("grant_type", "client_credentials"); + // form.add("scope", "PRINCIPAL_ROLE:ALL"); + // form.add("client_id", client_id); + // form.add("client_secret", client_secret); + + + LOG_TEST(log, "kssenii 4"); Poco::URI url(base_url / oauth_tokens_endpoint); - // url.setQueryParameters(params); + url.setQueryParameters(params); LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); @@ -211,7 +232,7 @@ bool RestCatalog::empty() const }; Namespaces namespaces; - getNamespacesRecursive("", namespaces, stop_condition); + getNamespacesRecursive("", namespaces, stop_condition, {}); return found_table; } @@ -224,19 +245,42 @@ bool RestCatalog::empty() const RestCatalog::Tables RestCatalog::getTables() const { - Namespaces namespaces; - getNamespacesRecursive("", namespaces, {}); + size_t num_threads = 10; + ThreadPool pool( + CurrentMetrics::MergeTreeDataSelectExecutorThreads, + CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, + CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled, + num_threads); + + DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); Tables tables; - for (const auto & current_namespace : namespaces) + std::mutex mutex; + + auto func = [&](const std::string & current_namespace) { - auto tables_in_namespace = getTables(current_namespace); - std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); - } + runner( + [&]{ + auto tables_in_namespace = getTables(current_namespace); + { + std::lock_guard lock(mutex); + std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); + } + }); + }; + + Namespaces namespaces; + getNamespacesRecursive("", namespaces, {}, func); + + runner.waitForAllToFinishAndRethrowFirstError(); return tables; } -void RestCatalog::getNamespacesRecursive(const std::string & base_namespace, Namespaces & result, StopCondition stop_condition) const +void RestCatalog::getNamespacesRecursive( + const std::string & base_namespace, + Namespaces & result, + StopCondition stop_condition, + ExecuteFunc func) const { auto namespaces = getNamespaces(base_namespace); result.reserve(result.size() + namespaces.size()); @@ -249,7 +293,10 @@ void RestCatalog::getNamespacesRecursive(const std::string & base_namespace, Nam if (stop_condition && stop_condition(current_namespace)) break; - getNamespacesRecursive(current_namespace, result, stop_condition); + if (func) + func(current_namespace); + + getNamespacesRecursive(current_namespace, result, stop_condition, func); } } diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 5415ff73a7e..067f67ab493 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -72,7 +72,13 @@ private: Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; using StopCondition = std::function; - void getNamespacesRecursive(const std::string & base_namespace, Namespaces & result, StopCondition stop_condition) const; + using ExecuteFunc = std::function; + + void getNamespacesRecursive( + const std::string & base_namespace, + Namespaces & result, + StopCondition stop_condition, + ExecuteFunc func) const; Namespaces getNamespaces(const std::string & base_namespace) const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index fd2fe0400bb..d7faf487188 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -80,7 +80,8 @@ StorageObjectStorage::StorageObjectStorage( std::optional format_settings_, LoadingStrictnessLevel mode, bool distributed_processing_, - ASTPtr partition_by_) + ASTPtr partition_by_, + bool lazy_init) : IStorage(table_id_) , configuration(configuration_) , object_storage(object_storage_) @@ -91,7 +92,8 @@ StorageObjectStorage::StorageObjectStorage( { try { - configuration->update(object_storage, context); + if (!lazy_init) + configuration->update(object_storage, context); } catch (...) { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index e2bb41a4935..0df0bbdefbd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -59,7 +59,8 @@ public: std::optional format_settings_, LoadingStrictnessLevel mode, bool distributed_processing_ = false, - ASTPtr partition_by_ = nullptr); + ASTPtr partition_by_ = nullptr, + bool lazy_init = false); String getName() const override; From 7b4aabbb718bdb10629e6e1ed659bfe7f1a33059 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Nov 2024 17:56:55 +0100 Subject: [PATCH 18/39] Support generating token on clickhouse server side --- src/Databases/Iceberg/RestCatalog.cpp | 40 ++++++++------------------- 1 file changed, 12 insertions(+), 28 deletions(-) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 76c5e897af9..b4ab45c3cac 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -75,13 +76,11 @@ RestCatalog::RestCatalog( /// TODO: remove before merge. LOG_TEST(log, "Client id: {}, client secret: {}", client_id, client_secret); } - LOG_TEST(log, "kssenii 1"); config = loadConfig(); } RestCatalog::Config RestCatalog::loadConfig() { - LOG_TEST(log, "kssenii 2"); if (!client_id.empty()) { static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; @@ -100,44 +99,29 @@ RestCatalog::Config RestCatalog::loadConfig() Poco::JSON::Stringifier::stringify(json, oss); std::string json_str = oss.str(); - Poco::URI::QueryParameters params = { - {"Content-Type", "application/x-www-form-urlencoded"}, - {"Content-Length", DB::toString(json_str.size())} - }; - // {"grant_type", "client_credentials"}, - // {"scope", "PRINCIPAL_ROLE:ALL"}, - // {"client_id", client_id}, - // {"client_secret", client_secret}}; + auto current_headers = headers; + current_headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); + current_headers.emplace_back("Accepts", "application/json; charset=UTF-8"); - - // LOG_TEST(log, "kssenii 3"); - - // DB::HTMLForm form(context->getSettingsRef()); - // form.add("grant_type", "client_credentials"); - // form.add("scope", "PRINCIPAL_ROLE:ALL"); - // form.add("client_id", client_id); - // form.add("client_secret", client_secret); - - - LOG_TEST(log, "kssenii 4"); Poco::URI url(base_url / oauth_tokens_endpoint); + Poco::URI::QueryParameters params = { + {"grant_type", "client_credentials"}, + {"scope", "PRINCIPAL_ROLE:ALL"}, + {"client_id", client_id}, + {"client_secret", client_secret}, + }; url.setQueryParameters(params); LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); - auto callback = [&](std::ostream & out) - { - out << json_str; - }; - auto wb = DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) .withSettings(getContext()->getReadSettings()) .withTimeouts(timeouts) .withHostFilter(&getContext()->getRemoteHostFilter()) - .withOutCallback(callback) .withSkipNotFound(false) - .withHeaders(headers) + .withHeaders(current_headers) .create(credentials); json_str.clear(); From 599d977d423b7c5d78f5d55789d43c1db1bd9832 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 19 Nov 2024 19:16:53 +0100 Subject: [PATCH 19/39] Update token lazily --- src/Databases/Iceberg/DatabaseIceberg.cpp | 14 +- src/Databases/Iceberg/DatabaseIceberg.h | 1 - src/Databases/Iceberg/RestCatalog.cpp | 208 +++++++++++++--------- src/Databases/Iceberg/RestCatalog.h | 7 +- 4 files changed, 133 insertions(+), 97 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index d701e73d429..6241c0d8423 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -78,18 +78,6 @@ DatabaseIceberg::DatabaseIceberg( , database_engine_definition(database_engine_definition_) , log(getLogger("DatabaseIceberg(" + database_name_ + ")")) { - const auto auth_header = settings[DatabaseIcebergSetting::auth_header].value; - LOG_TEST(log, "Auth header: {}", auth_header); - if (!auth_header.empty()) - { - auto pos = auth_header.find(':'); - if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); - headers.emplace_back(auth_header.substr(0, pos), auth_header.substr(pos + 1)); - - LOG_TEST(log, "Added header: {}={}", headers.back().name, headers.back().value); - } - validateSettings(context_); } @@ -128,7 +116,7 @@ std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const settings[DatabaseIcebergSetting::warehouse].value, url, settings[DatabaseIcebergSetting::catalog_credential].value, - headers, + settings[DatabaseIcebergSetting::auth_header], Context::getGlobalContextInstance()); } } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index a355e316c1a..145f033ad6a 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -58,7 +58,6 @@ private: const LoggerPtr log; /// Crendetials to authenticate Iceberg Catalog. Poco::Net::HTTPBasicCredentials credentials; - HTTPHeaderEntries headers; mutable std::shared_ptr catalog_impl; diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index b4ab45c3cac..d336c4af707 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -49,94 +49,49 @@ std::string RestCatalog::Config::toString() const return wb.str(); } -RestCatalog::RestCatalog( - const std::string & warehouse_, - const std::string & base_url_, - const std::string & catalog_credential_, - const DB::HTTPHeaderEntries & headers_, - DB::ContextPtr context_) - : ICatalog(warehouse_) - , DB::WithContext(context_) - , log(getLogger("RestCatalog(" + warehouse_ + ")")) - , base_url(base_url_) - , headers(headers_) +static std::pair parseCatalogCredential(const std::string & catalog_credential) { - if (!catalog_credential_.empty()) + std::string client_id, client_secret; + if (!catalog_credential.empty()) { - auto pos = catalog_credential_.find(':'); + auto pos = catalog_credential.find(':'); if (pos == std::string::npos) { throw DB::Exception( DB::ErrorCodes::BAD_ARGUMENTS, "Unexpected format of catalog credential: " "expected client_id and client_secret separated by `:`"); } - client_id = catalog_credential_.substr(0, pos); - client_secret = catalog_credential_.substr(pos + 1); + client_id = catalog_credential.substr(0, pos); + client_secret = catalog_credential.substr(pos + 1); + } + return std::pair(client_id, client_secret); +} - /// TODO: remove before merge. - LOG_TEST(log, "Client id: {}, client secret: {}", client_id, client_secret); +RestCatalog::RestCatalog( + const std::string & warehouse_, + const std::string & base_url_, + const std::string & catalog_credential_, + const std::string & auth_header_, + DB::ContextPtr context_) + : ICatalog(warehouse_) + , DB::WithContext(context_) + , log(getLogger("RestCatalog(" + warehouse_ + ")")) + , base_url(base_url_) +{ + std::tie(client_id, client_secret) = parseCatalogCredential(catalog_credential_); + if (!auth_header_.empty()) + { + auto pos = auth_header_.find(':'); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); + + auth_header = DB::HTTPHeaderEntry(auth_header_.substr(0, pos), auth_header_.substr(pos + 1)); } config = loadConfig(); } RestCatalog::Config RestCatalog::loadConfig() { - if (!client_id.empty()) - { - static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; - - const auto & context = getContext(); - const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); - - Poco::JSON::Object json; - json.set("grant_type", "client_credentials"); - json.set("scope", "PRINCIPAL_ROLE:ALL"); - json.set("client_id", client_id); - json.set("client_secret", client_secret); - - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss.exceptions(std::ios::failbit); - Poco::JSON::Stringifier::stringify(json, oss); - std::string json_str = oss.str(); - - auto current_headers = headers; - current_headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); - current_headers.emplace_back("Accepts", "application/json; charset=UTF-8"); - - Poco::URI url(base_url / oauth_tokens_endpoint); - Poco::URI::QueryParameters params = { - {"grant_type", "client_credentials"}, - {"scope", "PRINCIPAL_ROLE:ALL"}, - {"client_id", client_id}, - {"client_secret", client_secret}, - }; - url.setQueryParameters(params); - - LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); - - auto wb = DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withMethod(Poco::Net::HTTPRequest::HTTP_POST) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) - .withHostFilter(&getContext()->getRemoteHostFilter()) - .withSkipNotFound(false) - .withHeaders(current_headers) - .create(credentials); - - json_str.clear(); - readJSONObjectPossiblyInvalid(json_str, *wb); - - LOG_TEST(log, "Received token result: {}", json_str); - - Poco::JSON::Parser parser; - Poco::Dynamic::Var res_json = parser.parse(json_str); - const Poco::JSON::Object::Ptr & object = res_json.extract(); - - auto access_token = object->get("access_token").extract(); - headers.emplace_back("Authorization", "Bearer " + access_token); - } - Poco::URI::QueryParameters params = {{"warehouse", warehouse}}; auto buf = createReadBuffer(config_endpoint, params); @@ -173,6 +128,78 @@ void RestCatalog::parseConfig(const Poco::JSON::Object::Ptr & object, Config & r result.default_base_location = object->get("default-base-location").extract(); } +DB::HTTPHeaderEntries RestCatalog::getHeaders(bool update_token) const +{ + if (auth_header.has_value()) + { + return DB::HTTPHeaderEntries{auth_header.value()}; + } + + if (!access_token.has_value() || update_token) + { + access_token = retrieveAccessToken(); + } + + DB::HTTPHeaderEntries headers; + headers.emplace_back("Authorization", "Bearer " + access_token.value()); + return headers; +} + +std::string RestCatalog::retrieveAccessToken() const +{ + static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; + + const auto & context = getContext(); + const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + + Poco::JSON::Object json; + json.set("grant_type", "client_credentials"); + json.set("scope", "PRINCIPAL_ROLE:ALL"); + json.set("client_id", client_id); + json.set("client_secret", client_secret); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(json, oss); + std::string json_str = oss.str(); + + DB::HTTPHeaderEntries headers; + headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); + headers.emplace_back("Accepts", "application/json; charset=UTF-8"); + + Poco::URI url(base_url / oauth_tokens_endpoint); + Poco::URI::QueryParameters params = { + {"grant_type", "client_credentials"}, + {"scope", "PRINCIPAL_ROLE:ALL"}, + {"client_id", client_id}, + {"client_secret", client_secret}, + }; + url.setQueryParameters(params); + + LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); + + auto wb = DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSkipNotFound(false) + .withHeaders(headers) + .create(credentials); + + json_str.clear(); + readJSONObjectPossiblyInvalid(json_str, *wb); + + LOG_TEST(log, "Received token result: {}", json_str); + + Poco::JSON::Parser parser; + Poco::Dynamic::Var res_json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = res_json.extract(); + + return object->get("access_token").extract(); +} + std::optional RestCatalog::getStorageType() const { if (config.default_base_location.empty()) @@ -191,16 +218,35 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( if (!params.empty()) url.setQueryParameters(params); + auto headers = getHeaders(false); + LOG_TEST(log, "Requesting: {}", url.toString()); - return DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) - .withHostFilter(&getContext()->getRemoteHostFilter()) - .withSkipNotFound(false) - .withHeaders(headers) - .create(credentials); + try + { + return DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(headers) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withDelayInit(false) + .withSkipNotFound(false) + .create(credentials); + } + catch (...) + { + auto new_headers = getHeaders(true); + return DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(timeouts) + .withHeaders(new_headers) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withDelayInit(false) + .withSkipNotFound(false) + .create(credentials); + } } bool RestCatalog::empty() const diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 067f67ab493..116b4253684 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -22,7 +22,7 @@ public: const std::string & warehouse_, const std::string & base_url_, const std::string & catalog_credential_, - const DB::HTTPHeaderEntries & headers_, + const std::string & auth_header_, DB::ContextPtr context_); ~RestCatalog() override = default; @@ -57,7 +57,8 @@ private: }; const std::filesystem::path base_url; - DB::HTTPHeaderEntries headers; + std::optional auth_header; + mutable std::optional access_token; std::string client_id; std::string client_secret; Config config; @@ -94,6 +95,8 @@ private: TableMetadata & result) const; Config loadConfig(); + std::string retrieveAccessToken() const; + DB::HTTPHeaderEntries getHeaders(bool update_token = false) const; static void parseConfig(const Poco::JSON::Object::Ptr & object, Config & result); }; From fd5023e2a6f111e1027d7abb6831f23c056b4e80 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Nov 2024 17:32:46 +0100 Subject: [PATCH 20/39] Add comments --- src/Common/CurrentMetrics.cpp | 3 + src/Databases/Iceberg/DatabaseIceberg.cpp | 12 +-- src/Databases/Iceberg/RestCatalog.cpp | 124 ++++++++++++---------- src/Databases/Iceberg/RestCatalog.h | 24 +++-- 4 files changed, 94 insertions(+), 69 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index d073e477593..33f0e10156f 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -189,6 +189,9 @@ M(BuildVectorSimilarityIndexThreads, "Number of threads in the build vector similarity index thread pool.") \ M(BuildVectorSimilarityIndexThreadsActive, "Number of threads in the build vector similarity index thread pool running a task.") \ M(BuildVectorSimilarityIndexThreadsScheduled, "Number of queued or active jobs in the build vector similarity index thread pool.") \ + M(IcebergCatalogThreads, "Number of threads in the IcebergCatalog thread pool.") \ + M(IcebergCatalogThreadsActive, "Number of threads in the IcebergCatalog thread pool running a task.") \ + M(IcebergCatalogThreadsScheduled, "Number of queued or active jobs in the IcebergCatalog thread pool.") \ \ M(DiskPlainRewritableAzureDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \ M(DiskPlainRewritableAzureFileCount, "Number of file entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \ diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 6241c0d8423..c672d805e5e 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -25,9 +25,9 @@ namespace CurrentMetrics { - extern const Metric MergeTreeDataSelectExecutorThreads; - extern const Metric MergeTreeDataSelectExecutorThreadsActive; - extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; + extern const Metric IcebergCatalogThreads; + extern const Metric IcebergCatalogThreadsActive; + extern const Metric IcebergCatalogThreadsScheduled; } @@ -236,9 +236,9 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( auto iceberg_tables = catalog->getTables(); size_t num_threads = std::min(10, iceberg_tables.size()); ThreadPool pool( - CurrentMetrics::MergeTreeDataSelectExecutorThreads, - CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, - CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled, + CurrentMetrics::IcebergCatalogThreads, + CurrentMetrics::IcebergCatalogThreadsActive, + CurrentMetrics::IcebergCatalogThreadsScheduled, num_threads); DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index d336c4af707..0fe9e90e46f 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -26,9 +26,9 @@ namespace DB::ErrorCodes namespace CurrentMetrics { - extern const Metric MergeTreeDataSelectExecutorThreads; - extern const Metric MergeTreeDataSelectExecutorThreadsActive; - extern const Metric MergeTreeDataSelectExecutorThreadsScheduled; + extern const Metric IcebergCatalogThreads; + extern const Metric IcebergCatalogThreadsActive; + extern const Metric IcebergCatalogThreadsScheduled; } namespace Iceberg @@ -37,20 +37,14 @@ namespace Iceberg static constexpr auto config_endpoint = "config"; static constexpr auto namespaces_endpoint = "namespaces"; -std::string RestCatalog::Config::toString() const +namespace { - DB::WriteBufferFromOwnString wb; - if (!prefix.empty()) - wb << "prefix: " << prefix.string() << ", "; - if (!default_base_location.empty()) - wb << "default_base_location: " << default_base_location << ", "; - - return wb.str(); -} - -static std::pair parseCatalogCredential(const std::string & catalog_credential) +std::pair parseCatalogCredential(const std::string & catalog_credential) { + /// Parse a string of format ":" + /// into separare strings client_id and client_secret. + std::string client_id, client_secret; if (!catalog_credential.empty()) { @@ -67,6 +61,33 @@ static std::pair parseCatalogCredential(const std::str return std::pair(client_id, client_secret); } +DB::HTTPHeaderEntry parseAuthHeader(const std::string & auth_header) +{ + /// Parse a string of format "Authorization: " + /// into a key-value header "Authorization", " " + + auto pos = auth_header.find(':'); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); + + return DB::HTTPHeaderEntry(auth_header.substr(0, pos), auth_header.substr(pos + 1)); +} + +} + +std::string RestCatalog::Config::toString() const +{ + DB::WriteBufferFromOwnString wb; + + if (!prefix.empty()) + wb << "prefix: " << prefix.string() << ", "; + + if (!default_base_location.empty()) + wb << "default_base_location: " << default_base_location << ", "; + + return wb.str(); +} + RestCatalog::RestCatalog( const std::string & warehouse_, const std::string & base_url_, @@ -75,18 +96,14 @@ RestCatalog::RestCatalog( DB::ContextPtr context_) : ICatalog(warehouse_) , DB::WithContext(context_) - , log(getLogger("RestCatalog(" + warehouse_ + ")")) , base_url(base_url_) + , log(getLogger("RestCatalog(" + warehouse_ + ")")) { - std::tie(client_id, client_secret) = parseCatalogCredential(catalog_credential_); - if (!auth_header_.empty()) - { - auto pos = auth_header_.find(':'); - if (pos == std::string::npos) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unexpected format of auth header"); + if (!catalog_credential_.empty()) + std::tie(client_id, client_secret) = parseCatalogCredential(catalog_credential_); + else if (!auth_header_.empty()) + auth_header = parseAuthHeader(auth_header_); - auth_header = DB::HTTPHeaderEntry(auth_header_.substr(0, pos), auth_header_.substr(pos + 1)); - } config = loadConfig(); } @@ -98,7 +115,7 @@ RestCatalog::Config RestCatalog::loadConfig() std::string json_str; readJSONObjectPossiblyInvalid(json_str, *buf); - LOG_TEST(log, "Received config result: {}", json_str); + LOG_TEST(log, "Received catalog configuration settings: {}", json_str); Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(json_str); @@ -107,16 +124,16 @@ RestCatalog::Config RestCatalog::loadConfig() Config result; auto defaults_object = object->get("defaults").extract(); - parseConfig(defaults_object, result); + parseCatalogConfigurationSettings(defaults_object, result); auto overrides_object = object->get("overrides").extract(); - parseConfig(overrides_object, result); + parseCatalogConfigurationSettings(overrides_object, result); - LOG_TEST(log, "Parsed config: {}", result.toString()); + LOG_TEST(log, "Parsed catalog configuration settings: {}", result.toString()); return result; } -void RestCatalog::parseConfig(const Poco::JSON::Object::Ptr & object, Config & result) +void RestCatalog::parseCatalogConfigurationSettings(const Poco::JSON::Object::Ptr & object, Config & result) { if (!object) return; @@ -135,34 +152,35 @@ DB::HTTPHeaderEntries RestCatalog::getHeaders(bool update_token) const return DB::HTTPHeaderEntries{auth_header.value()}; } - if (!access_token.has_value() || update_token) + if (!client_id.empty()) { - access_token = retrieveAccessToken(); + if (!access_token.has_value() || update_token) + { + access_token = retrieveAccessToken(); + } + + DB::HTTPHeaderEntries headers; + headers.emplace_back("Authorization", "Bearer " + access_token.value()); + return headers; } - DB::HTTPHeaderEntries headers; - headers.emplace_back("Authorization", "Bearer " + access_token.value()); - return headers; + return {}; } std::string RestCatalog::retrieveAccessToken() const { static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; - const auto & context = getContext(); - const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); + /// TODO: + /// 1. support oauth2-server-uri + /// https://github.com/apache/iceberg/blob/918f81f3c3f498f46afcea17c1ac9cdc6913cb5c/open-api/rest-catalog-open-api.yaml#L183C82-L183C99 Poco::JSON::Object json; json.set("grant_type", "client_credentials"); - json.set("scope", "PRINCIPAL_ROLE:ALL"); + json.set("scope", "PRINCIPAL_ROLE:ALL"); /// TODO: add it into setting. json.set("client_id", client_id); json.set("client_secret", client_secret); - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss.exceptions(std::ios::failbit); - Poco::JSON::Stringifier::stringify(json, oss); - std::string json_str = oss.str(); - DB::HTTPHeaderEntries headers; headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); headers.emplace_back("Accepts", "application/json; charset=UTF-8"); @@ -176,23 +194,20 @@ std::string RestCatalog::retrieveAccessToken() const }; url.setQueryParameters(params); - LOG_TEST(log, "Writing {}: {}", url.toString(), json_str); - + const auto & context = getContext(); auto wb = DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) .withMethod(Poco::Net::HTTPRequest::HTTP_POST) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) - .withHostFilter(&getContext()->getRemoteHostFilter()) + .withSettings(context->getReadSettings()) + .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) + .withHostFilter(&context->getRemoteHostFilter()) .withSkipNotFound(false) .withHeaders(headers) .create(credentials); - json_str.clear(); + std::string json_str; readJSONObjectPossiblyInvalid(json_str, *wb); - LOG_TEST(log, "Received token result: {}", json_str); - Poco::JSON::Parser parser; Poco::Dynamic::Var res_json = parser.parse(json_str); const Poco::JSON::Object::Ptr & object = res_json.extract(); @@ -212,7 +227,6 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( const Poco::URI::QueryParameters & params) const { const auto & context = getContext(); - const auto timeouts = DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings()); Poco::URI url(base_url / endpoint); if (!params.empty()) @@ -227,7 +241,7 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( return DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) + .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) .withHeaders(headers) .withHostFilter(&getContext()->getRemoteHostFilter()) .withDelayInit(false) @@ -240,7 +254,7 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( return DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) .withSettings(getContext()->getReadSettings()) - .withTimeouts(timeouts) + .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) .withHeaders(new_headers) .withHostFilter(&getContext()->getRemoteHostFilter()) .withDelayInit(false) @@ -277,9 +291,9 @@ RestCatalog::Tables RestCatalog::getTables() const { size_t num_threads = 10; ThreadPool pool( - CurrentMetrics::MergeTreeDataSelectExecutorThreads, - CurrentMetrics::MergeTreeDataSelectExecutorThreadsActive, - CurrentMetrics::MergeTreeDataSelectExecutorThreadsScheduled, + CurrentMetrics::IcebergCatalogThreads, + CurrentMetrics::IcebergCatalogThreadsActive, + CurrentMetrics::IcebergCatalogThreadsScheduled, num_threads); DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 116b4253684..6c842e1dff7 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -46,25 +46,33 @@ public: std::optional getStorageType() const override; private: - LoggerPtr log; - struct Config { + /// Prefix is a path of the catalog enpoint, + /// e.g. /v1/{prefix}/namespaces/{namespace}/tables/{table} std::filesystem::path prefix; + /// Base location is location of data in storage + /// (in filesystem or object storage). std::string default_base_location; std::string toString() const; }; const std::filesystem::path base_url; - std::optional auth_header; - mutable std::optional access_token; - std::string client_id; - std::string client_secret; + const LoggerPtr log; + + /// Catalog configuration settings from /v1/config endpoint. Config config; - Poco::Net::HTTPBasicCredentials credentials{}; + /// Auth headers of format: "Authorization": " " + std::optional auth_header; + /// Parameters for OAuth. + std::string client_id; + std::string client_secret; + mutable std::optional access_token; + + Poco::Net::HTTPBasicCredentials credentials{}; DB::ReadWriteBufferFromHTTPPtr createReadBuffer( const std::string & endpoint, @@ -97,7 +105,7 @@ private: Config loadConfig(); std::string retrieveAccessToken() const; DB::HTTPHeaderEntries getHeaders(bool update_token = false) const; - static void parseConfig(const Poco::JSON::Object::Ptr & object, Config & result); + static void parseCatalogConfigurationSettings(const Poco::JSON::Object::Ptr & object, Config & result); }; } From 4abbf29a865f0805dd125a188f7e312ba67400a1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 20 Nov 2024 18:59:47 +0100 Subject: [PATCH 21/39] Fix style check --- src/Databases/Iceberg/RestCatalog.cpp | 1 + src/Databases/Iceberg/RestCatalog.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 0fe9e90e46f..a34aa45b812 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -22,6 +22,7 @@ namespace DB::ErrorCodes { extern const int ICEBERG_CATALOG_ERROR; extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } namespace CurrentMetrics diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 6c842e1dff7..402e761b07a 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -48,7 +48,7 @@ public: private: struct Config { - /// Prefix is a path of the catalog enpoint, + /// Prefix is a path of the catalog endpoint, /// e.g. /v1/{prefix}/namespaces/{namespace}/tables/{table} std::filesystem::path prefix; /// Base location is location of data in storage From 1016add3d8cddc06285f02da0fc7dfcf44543b9d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 14:22:54 +0100 Subject: [PATCH 22/39] Add comments, fix test --- src/Databases/Iceberg/DatabaseIceberg.cpp | 2 +- src/Databases/Iceberg/ICatalog.cpp | 15 ------- src/Databases/Iceberg/ICatalog.h | 33 ++++++++++----- src/Databases/Iceberg/RestCatalog.cpp | 40 +++++++++++++------ src/Databases/Iceberg/RestCatalog.h | 6 +-- .../integration/test_database_iceberg/test.py | 5 ++- 6 files changed, 58 insertions(+), 43 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index c672d805e5e..6f06d14d30d 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -249,7 +249,7 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( if (filter_by_table_name && !filter_by_table_name(table_name)) continue; - runner([&]{ + runner([=, &tables, &mutexx, this]{ auto storage = tryGetTable(table_name, context_); { std::lock_guard lock(mutexx); diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index 6b6ecc85daf..db8ad443c0e 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -68,19 +68,4 @@ const DB::NamesAndTypesList & TableMetadata::getSchema() const return schema; } -StorageType ICatalog::getStorageType(const std::string & location) -{ - auto pos = location.find("://"); - if (pos == std::string::npos) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); - - auto storage_type_str = location.substr(0, pos); - auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); - - if (!storage_type) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_str); - - return *storage_type; -} - } diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index 4c61c87998d..aba1884ab3e 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -7,6 +7,8 @@ namespace Iceberg { using StorageType = DB::DatabaseIcebergStorageType; +/// A class representing table metadata, +/// which was received from Catalog. class TableMetadata { public: @@ -15,23 +17,21 @@ public: TableMetadata & withLocation() { with_location = true; return *this; } TableMetadata & withSchema() { with_schema = true; return *this; } + void setLocation(const std::string & location_); std::string getLocation(bool path_only) const; - std::string getLocation() const; - std::string getLocationWithoutPath() const; + void setSchema(const DB::NamesAndTypesList & schema_); const DB::NamesAndTypesList & getSchema() const; bool requiresLocation() const { return with_location; } bool requiresSchema() const { return with_schema; } - void setLocation(const std::string & location_); - void setSchema(const DB::NamesAndTypesList & schema_); - private: - /// starts with s3://, file://, etc + /// Starts with s3://, file://, etc. + /// For example, `s3://bucket/` std::string location_without_path; + /// Path to table's data: `/path/to/table/data/` std::string path; - /// column names and types DB::NamesAndTypesList schema; bool with_location = false; @@ -39,40 +39,51 @@ private: }; +/// Base class for catalog implementation. +/// Used for communication with the catalog. class ICatalog { public: using Namespaces = std::vector; - using Tables = std::vector; explicit ICatalog(const std::string & warehouse_) : warehouse(warehouse_) {} virtual ~ICatalog() = default; + /// Does catalog have any tables? virtual bool empty() const = 0; - virtual Tables getTables() const = 0; + /// Fetch tables' names list. + /// Contains full namespaces in names. + virtual DB::Names getTables() const = 0; + /// Check that a table exists in a given namespace. virtual bool existsTable( const std::string & namespace_naem, const std::string & table_name) const = 0; + /// Get table metadata in the given namespace. + /// Throw exception if table does not exist. virtual void getTableMetadata( const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const = 0; + /// Get table metadata in the given namespace. + /// Return `false` if table does not exist, `true` otherwise. virtual bool tryGetTableMetadata( const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const = 0; + /// Get storage type, where Iceberg tables' data is stored. + /// E.g. one of S3, Azure, Local, HDFS. virtual std::optional getStorageType() const = 0; protected: + /// Name of the warehouse, + /// which is sometimes also called "catalog name". const std::string warehouse; - - static StorageType getStorageType(const std::string & location); }; } diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index a34aa45b812..345222fcea8 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -74,6 +74,21 @@ DB::HTTPHeaderEntry parseAuthHeader(const std::string & auth_header) return DB::HTTPHeaderEntry(auth_header.substr(0, pos), auth_header.substr(pos + 1)); } +StorageType parseStorageTypeFromLocation(const std::string & location) +{ + auto pos = location.find("://"); + if (pos == std::string::npos) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); + + auto storage_type_str = location.substr(0, pos); + auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); + + if (!storage_type) + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_str); + + return *storage_type; +} + } std::string RestCatalog::Config::toString() const @@ -220,7 +235,7 @@ std::optional RestCatalog::getStorageType() const { if (config.default_base_location.empty()) return std::nullopt; - return ICatalog::getStorageType(config.default_base_location); + return parseStorageTypeFromLocation(config.default_base_location); } DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( @@ -288,7 +303,7 @@ bool RestCatalog::empty() const } } -RestCatalog::Tables RestCatalog::getTables() const +DB::Names RestCatalog::getTables() const { size_t num_threads = 10; ThreadPool pool( @@ -299,18 +314,16 @@ RestCatalog::Tables RestCatalog::getTables() const DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); - Tables tables; + DB::Names tables; std::mutex mutex; auto func = [&](const std::string & current_namespace) { runner( - [&]{ + [=, &tables, &mutex, this]{ auto tables_in_namespace = getTables(current_namespace); - { - std::lock_guard lock(mutex); - std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); - } + std::lock_guard lock(mutex); + std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); }); }; @@ -369,7 +382,7 @@ RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_name { auto buf = createReadBuffer(config.prefix / namespaces_endpoint, params); auto namespaces = parseNamespaces(*buf, base_namespace); - LOG_TEST(log, "Loaded {} namespaces", namespaces.size()); + LOG_TEST(log, "Loaded {} namespaces in base namespace {}", namespaces.size(), base_namespace); return namespaces; } catch (const DB::HTTPException & e) @@ -426,14 +439,14 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const return namespaces; } -RestCatalog::Tables RestCatalog::getTables(const std::string & base_namespace, size_t limit) const +DB::Names RestCatalog::getTables(const std::string & base_namespace, size_t limit) const { const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; auto buf = createReadBuffer(config.prefix / endpoint); return parseTables(*buf, base_namespace, limit); } -RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const +DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const { if (buf.eof()) return {}; @@ -449,12 +462,15 @@ RestCatalog::Tables RestCatalog::parseTables(DB::ReadBuffer & buf, const std::st if (!identifiers_object) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); - Tables tables; + DB::Names tables; for (size_t i = 0; i < identifiers_object->size(); ++i) { const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); const auto table_name = current_table_json->get("name").extract(); + LOG_TEST(log, "Base namespace: {}", base_namespace); + LOG_TEST(log, "Table name: {}", table_name); + tables.push_back(base_namespace + "." + table_name); if (limit && tables.size() >= limit) break; diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 402e761b07a..5e92ec846a0 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -29,7 +29,7 @@ public: bool empty() const override; - Tables getTables() const override; + DB::Names getTables() const override; bool existsTable(const std::string & namespace_name, const std::string & table_name) const override; @@ -93,9 +93,9 @@ private: Namespaces parseNamespaces(DB::ReadBuffer & buf, const std::string & base_namespace) const; - Tables getTables(const std::string & base_namespace, size_t limit = 0) const; + DB::Names getTables(const std::string & base_namespace, size_t limit = 0) const; - Tables parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const; + DB::Names parseTables(DB::ReadBuffer & buf, const std::string & base_namespace, size_t limit) const; bool getTableMetadataImpl( const std::string & namespace_name, diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 51d89444c2e..c0f0087522d 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -122,7 +122,10 @@ def create_clickhouse_iceberg_database(started_cluster, node, name): f""" DROP DATABASE IF EXISTS {name}; CREATE DATABASE {name} ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') -SETTINGS catalog_type = 'rest', storage_endpoint = 'http://minio:9000/' +SETTINGS catalog_type = 'rest', + storage_endpoint = 'http://minio:9000/warehouse', + warehouse='demo', + storage_type='s3' """ ) From 8393e4382ca490c98624fc9c46ceec45bddb48f5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 17:36:50 +0100 Subject: [PATCH 23/39] Better --- src/Core/ServerSettings.cpp | 2 + src/Databases/Iceberg/DatabaseIceberg.cpp | 14 +- src/Databases/Iceberg/DatabaseIceberg.h | 4 - .../Iceberg/DatabaseIcebergSettings.cpp | 1 + src/Databases/Iceberg/ICatalog.h | 2 + src/Databases/Iceberg/RestCatalog.cpp | 126 +++++++++++------- src/Databases/Iceberg/RestCatalog.h | 3 + src/Interpreters/Context.cpp | 24 ++++ src/Interpreters/Context.h | 2 + 9 files changed, 115 insertions(+), 63 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index 4bea23d4e90..47424986c9e 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -205,6 +205,8 @@ namespace DB DECLARE(UInt64, load_marks_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into prefetches pool", 0) \ DECLARE(UInt64, threadpool_writer_pool_size, 100, "Size of background pool for write requests to object storages", 0) \ DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \ + DECLARE(UInt64, iceberg_catalog_threadpool_pool_size, 50, "Size of background pool for iceberg catalog", 0) \ + DECLARE(UInt64, iceberg_catalog_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into iceberg catalog pool", 0) \ DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 6f06d14d30d..2d096ed4d64 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -40,6 +40,7 @@ namespace DatabaseIcebergSetting extern const DatabaseIcebergSettingsString warehouse; extern const DatabaseIcebergSettingsString catalog_credential; extern const DatabaseIcebergSettingsString auth_header; + extern const DatabaseIcebergSettingsString auth_scope; extern const DatabaseIcebergSettingsString storage_endpoint; } @@ -116,6 +117,7 @@ std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const settings[DatabaseIcebergSetting::warehouse].value, url, settings[DatabaseIcebergSetting::catalog_credential].value, + settings[DatabaseIcebergSetting::auth_scope].value, settings[DatabaseIcebergSetting::auth_header], Context::getGlobalContextInstance()); } @@ -125,6 +127,8 @@ std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const std::shared_ptr DatabaseIceberg::getConfiguration() const { + /// TODO: add tests for azure, local storage types. + switch (settings[DatabaseIcebergSetting::storage_type].value) { #if USE_AWS_S3 @@ -232,15 +236,9 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( { Tables tables; auto catalog = getCatalog(context_); + const auto iceberg_tables = catalog->getTables(); - auto iceberg_tables = catalog->getTables(); - size_t num_threads = std::min(10, iceberg_tables.size()); - ThreadPool pool( - CurrentMetrics::IcebergCatalogThreads, - CurrentMetrics::IcebergCatalogThreadsActive, - CurrentMetrics::IcebergCatalogThreadsScheduled, - num_threads); - + auto & pool = getContext()->getIcebergCatalogThreadpool(); DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); std::mutex mutexx; diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 145f033ad6a..4b9929a3a76 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -11,10 +11,6 @@ namespace DB { -/// TODO: -/// - auth: oauth, bearer token? -/// - tests with azure, hdfs, local - class DatabaseIceberg final : public IDatabase, WithContext { public: diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index 1dc1ba1f61c..d5c13d69f9d 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ DECLARE(String, catalog_credential, "", "", 0) \ + DECLARE(String, auth_scope, "PRINCIPAL_ROLE:ALL", "Authorization scope for client credentials or token exchange", 0) \ DECLARE(String, warehouse, "", "Warehouse name inside the catalog", 0) \ DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index aba1884ab3e..aa574509c8a 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -34,6 +34,8 @@ private: std::string path; DB::NamesAndTypesList schema; + std::string credentials; + bool with_location = false; bool with_schema = false; }; diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 345222fcea8..149a26f107a 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -25,13 +25,6 @@ namespace DB::ErrorCodes extern const int BAD_ARGUMENTS; } -namespace CurrentMetrics -{ - extern const Metric IcebergCatalogThreads; - extern const Metric IcebergCatalogThreadsActive; - extern const Metric IcebergCatalogThreadsScheduled; -} - namespace Iceberg { @@ -76,15 +69,26 @@ DB::HTTPHeaderEntry parseAuthHeader(const std::string & auth_header) StorageType parseStorageTypeFromLocation(const std::string & location) { + /// Table location in catalog metadata always starts with one of s3://, file://, etc. + /// So just extract this part of the path and deduce storage type from it. + auto pos = location.find("://"); if (pos == std::string::npos) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); + { + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, + "Unexpected path format: {}", location); + } auto storage_type_str = location.substr(0, pos); auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); if (!storage_type) - throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_str); + { + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, + "Unsupported storage type: {}", storage_type_str); + } return *storage_type; } @@ -108,15 +112,20 @@ RestCatalog::RestCatalog( const std::string & warehouse_, const std::string & base_url_, const std::string & catalog_credential_, + const std::string & auth_scope_, const std::string & auth_header_, DB::ContextPtr context_) : ICatalog(warehouse_) , DB::WithContext(context_) , base_url(base_url_) , log(getLogger("RestCatalog(" + warehouse_ + ")")) + , auth_scope(auth_scope_) { if (!catalog_credential_.empty()) + { std::tie(client_id, client_secret) = parseCatalogCredential(catalog_credential_); + update_token_if_expired = true; + } else if (!auth_header_.empty()) auth_header = parseAuthHeader(auth_header_); @@ -163,11 +172,16 @@ void RestCatalog::parseCatalogConfigurationSettings(const Poco::JSON::Object::Pt DB::HTTPHeaderEntries RestCatalog::getHeaders(bool update_token) const { + /// Option 1: user specified auth header manually. + /// Header has format: 'Authorization: '. if (auth_header.has_value()) { return DB::HTTPHeaderEntries{auth_header.value()}; } + /// Option 2: user provided grant_type, client_id and client_secret. + /// We would make OAuthClientCredentialsRequest + /// https://github.com/apache/iceberg/blob/3badfe0c1fcf0c0adfc7aa4a10f0b50365c48cf9/open-api/rest-catalog-open-api.yaml#L3498C5-L3498C34 if (!client_id.empty()) { if (!access_token.has_value() || update_token) @@ -191,12 +205,6 @@ std::string RestCatalog::retrieveAccessToken() const /// 1. support oauth2-server-uri /// https://github.com/apache/iceberg/blob/918f81f3c3f498f46afcea17c1ac9cdc6913cb5c/open-api/rest-catalog-open-api.yaml#L183C82-L183C99 - Poco::JSON::Object json; - json.set("grant_type", "client_credentials"); - json.set("scope", "PRINCIPAL_ROLE:ALL"); /// TODO: add it into setting. - json.set("client_id", client_id); - json.set("client_secret", client_secret); - DB::HTTPHeaderEntries headers; headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); headers.emplace_back("Accepts", "application/json; charset=UTF-8"); @@ -204,7 +212,7 @@ std::string RestCatalog::retrieveAccessToken() const Poco::URI url(base_url / oauth_tokens_endpoint); Poco::URI::QueryParameters params = { {"grant_type", "client_credentials"}, - {"scope", "PRINCIPAL_ROLE:ALL"}, + {"scope", auth_scope}, {"client_id", client_id}, {"client_secret", client_secret}, }; @@ -248,34 +256,35 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( if (!params.empty()) url.setQueryParameters(params); - auto headers = getHeaders(false); + auto create_buffer = [&](bool update_token) + { + auto headers = getHeaders(update_token); + return DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withSettings(getContext()->getReadSettings()) + .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) + .withHostFilter(&getContext()->getRemoteHostFilter()) + .withHeaders(headers) + .withDelayInit(false) + .withSkipNotFound(false) + .create(credentials); + }; LOG_TEST(log, "Requesting: {}", url.toString()); try { - return DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) - .withHeaders(headers) - .withHostFilter(&getContext()->getRemoteHostFilter()) - .withDelayInit(false) - .withSkipNotFound(false) - .create(credentials); + return create_buffer(false); } - catch (...) + catch (const DB::HTTPException & e) { - auto new_headers = getHeaders(true); - return DB::BuilderRWBufferFromHTTP(url) - .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) - .withSettings(getContext()->getReadSettings()) - .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) - .withHeaders(new_headers) - .withHostFilter(&getContext()->getRemoteHostFilter()) - .withDelayInit(false) - .withSkipNotFound(false) - .create(credentials); + if (update_token_if_expired && + (e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED + || e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN)) + { + return create_buffer(true); + } + throw; } } @@ -292,7 +301,7 @@ bool RestCatalog::empty() const }; Namespaces namespaces; - getNamespacesRecursive("", namespaces, stop_condition, {}); + getNamespacesRecursive("", namespaces, stop_condition, /* execute_func */{}); return found_table; } @@ -305,22 +314,17 @@ bool RestCatalog::empty() const DB::Names RestCatalog::getTables() const { - size_t num_threads = 10; - ThreadPool pool( - CurrentMetrics::IcebergCatalogThreads, - CurrentMetrics::IcebergCatalogThreadsActive, - CurrentMetrics::IcebergCatalogThreadsScheduled, - num_threads); - + auto & pool = getContext()->getIcebergCatalogThreadpool(); DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); DB::Names tables; std::mutex mutex; - auto func = [&](const std::string & current_namespace) + auto execute_for_each_namespace = [&](const std::string & current_namespace) { runner( - [=, &tables, &mutex, this]{ + [=, &tables, &mutex, this] + { auto tables_in_namespace = getTables(current_namespace); std::lock_guard lock(mutex); std::move(tables_in_namespace.begin(), tables_in_namespace.end(), std::back_inserter(tables)); @@ -328,7 +332,11 @@ DB::Names RestCatalog::getTables() const }; Namespaces namespaces; - getNamespacesRecursive("", namespaces, {}, func); + getNamespacesRecursive( + /* base_namespace */"", /// Empty base namespace means starting from root. + namespaces, + /* stop_condition */{}, + /* execute_func */execute_for_each_namespace); runner.waitForAllToFinishAndRethrowFirstError(); return tables; @@ -468,9 +476,6 @@ DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & bas const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); const auto table_name = current_table_json->get("name").extract(); - LOG_TEST(log, "Base namespace: {}", base_namespace); - LOG_TEST(log, "Table name: {}", table_name); - tables.push_back(base_namespace + "." + table_name); if (limit && tables.size() >= limit) break; @@ -550,6 +555,25 @@ bool RestCatalog::getTableMetadataImpl( int format_version = metadata_object->getValue("format-version"); result.setSchema(DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first); } + + // if (result.requiresCredentials()) + // { + // try + // { + // const auto credentials_endpoint = std::filesystem::path(endpoint) / "credentials"; + // auto credentials_buf = createReadBuffer(config.prefix / credentials_endpoint); + + // String credentials_json_str; + // readJSONObjectPossiblyInvalid(credentials_json_str, *credentials_buf); + + // LOG_TEST(log, "Credentials : {}", credentials_json_str); + // } + // catch (...) + // { + // DB::tryLogCurrentException(log); + // } + // } + return true; } diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 5e92ec846a0..570707e3aa0 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -22,6 +22,7 @@ public: const std::string & warehouse_, const std::string & base_url_, const std::string & catalog_credential_, + const std::string & auth_scope_, const std::string & auth_header_, DB::ContextPtr context_); @@ -68,8 +69,10 @@ private: std::optional auth_header; /// Parameters for OAuth. + bool update_token_if_expired = false; std::string client_id; std::string client_secret; + std::string auth_scope; mutable std::optional access_token; Poco::Net::HTTPBasicCredentials credentials{}; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 7f0ad013c1d..45aa69eb93d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -174,6 +174,9 @@ namespace CurrentMetrics extern const Metric AttachedDictionary; extern const Metric AttachedDatabase; extern const Metric PartsActive; + extern const Metric IcebergCatalogThreads; + extern const Metric IcebergCatalogThreadsActive; + extern const Metric IcebergCatalogThreadsScheduled; } @@ -282,6 +285,8 @@ namespace ServerSetting extern const ServerSettingsUInt64 load_marks_threadpool_queue_size; extern const ServerSettingsUInt64 threadpool_writer_pool_size; extern const ServerSettingsUInt64 threadpool_writer_queue_size; + extern const ServerSettingsUInt64 iceberg_catalog_threadpool_pool_size; + extern const ServerSettingsUInt64 iceberg_catalog_threadpool_queue_size; } @@ -412,6 +417,8 @@ struct ContextSharedPart : boost::noncopyable mutable std::unique_ptr load_marks_threadpool; /// Threadpool for loading marks cache. mutable OnceFlag prefetch_threadpool_initialized; mutable std::unique_ptr prefetch_threadpool; /// Threadpool for loading marks cache. + mutable std::unique_ptr iceberg_catalog_threadpool; + mutable OnceFlag iceberg_catalog_threadpool_initialized; mutable OnceFlag build_vector_similarity_index_threadpool_initialized; mutable std::unique_ptr build_vector_similarity_index_threadpool; /// Threadpool for vector-similarity index creation. mutable UncompressedCachePtr index_uncompressed_cache TSA_GUARDED_BY(mutex); /// The cache of decompressed blocks for MergeTree indices. @@ -3254,6 +3261,23 @@ ThreadPool & Context::getLoadMarksThreadpool() const return *shared->load_marks_threadpool; } +ThreadPool & Context::getIcebergCatalogThreadpool() const +{ + callOnce(shared->iceberg_catalog_threadpool_initialized, [&] + { + auto pool_size = shared->server_settings[ServerSetting::iceberg_catalog_threadpool_pool_size]; + auto queue_size = shared->server_settings[ServerSetting::iceberg_catalog_threadpool_queue_size]; + + shared->iceberg_catalog_threadpool = std::make_unique( + CurrentMetrics::IcebergCatalogThreads, + CurrentMetrics::IcebergCatalogThreadsActive, + CurrentMetrics::IcebergCatalogThreadsScheduled, + pool_size, pool_size, queue_size); + }); + + return *shared->iceberg_catalog_threadpool; +} + void Context::setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio) { std::lock_guard lock(shared->mutex); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 327ac0af5fd..f97bd055890 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1076,6 +1076,8 @@ public: void clearMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; + ThreadPool & getIcebergCatalogThreadpool() const; + void setIndexUncompressedCache(const String & cache_policy, size_t max_size_in_bytes, double size_ratio); void updateIndexUncompressedCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getIndexUncompressedCache() const; From c22e45dff7e2f4ebaeaab0b25cd7b90b1105cba0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 18:59:41 +0100 Subject: [PATCH 24/39] Support vended credentials --- src/Databases/Iceberg/DatabaseIceberg.cpp | 29 +++++++- .../Iceberg/DatabaseIcebergSettings.cpp | 1 + .../Iceberg/DatabaseIcebergSettings.h | 1 + src/Databases/Iceberg/ICatalog.cpp | 15 +++++ src/Databases/Iceberg/ICatalog.h | 11 +++- src/Databases/Iceberg/RestCatalog.cpp | 66 +++++++++++++------ src/Databases/Iceberg/RestCatalog.h | 5 +- src/Databases/Iceberg/StorageCredentials.h | 50 ++++++++++++++ 8 files changed, 152 insertions(+), 26 deletions(-) create mode 100644 src/Databases/Iceberg/StorageCredentials.h diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 2d096ed4d64..5223bf14452 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -42,6 +42,7 @@ namespace DatabaseIcebergSetting extern const DatabaseIcebergSettingsString auth_header; extern const DatabaseIcebergSettingsString auth_scope; extern const DatabaseIcebergSettingsString storage_endpoint; + extern const DatabaseIcebergSettingsBool vended_credentials; } namespace ErrorCodes @@ -192,6 +193,11 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ { auto catalog = getCatalog(context_); auto table_metadata = Iceberg::TableMetadata().withLocation().withSchema(); + + const bool with_vended_credentials = settings[DatabaseIcebergSetting::vended_credentials].value; + if (with_vended_credentials) + table_metadata = table_metadata.withStorageCredentials(); + auto [namespace_name, table_name] = parseTableName(name); if (!catalog->tryGetTableMetadata(namespace_name, table_name, table_metadata)) @@ -205,6 +211,25 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ auto table_endpoint = getStorageEndpointForTable(table_metadata); args[0] = std::make_shared(table_endpoint); + /// We either fetch storage credentials from catalog " + /// "or get storage credentials from database engine arguments + /// in CREATE query (e.g. in `args`). + /// Vended credentials can be disabled in catalog itself, + /// so we have a separate setting to know whether we should even try to fetch them. + if (with_vended_credentials && args.size() == 1) + { + auto storage_credentials = table_metadata.getStorageCredentials(); + if (storage_credentials) + storage_credentials->addCredentialsToEngineArgs(args); + } + else if (args.size() == 1) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Either vended credentials need to be enabled " + "or storage credentials need to be specified in database engine arguements in CREATE query"); + } + LOG_TEST(log, "Using table endpoint: {}", table_endpoint); const auto columns = ColumnsDescription(table_metadata.getSchema()); @@ -238,7 +263,7 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( auto catalog = getCatalog(context_); const auto iceberg_tables = catalog->getTables(); - auto & pool = getContext()->getIcebergCatalogThreadpool(); + auto & pool = context_->getIcebergCatalogThreadpool(); DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); std::mutex mutexx; @@ -335,7 +360,7 @@ void registerDatabaseIceberg(DatabaseFactory & factory) if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); - const size_t max_args_num = 3; + const size_t max_args_num = 1; if (engine_args.size() != max_args_num) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine must have {} arguments", max_args_num); diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index d5c13d69f9d..de04fc0bd11 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ DECLARE(String, catalog_credential, "", "", 0) \ + DECLARE(Bool, vended_credentials, true, "Use vended credentials (storage credentials) from catalog", 0) \ DECLARE(String, auth_scope, "PRINCIPAL_ROLE:ALL", "Authorization scope for client credentials or token exchange", 0) \ DECLARE(String, warehouse, "", "Warehouse name inside the catalog", 0) \ DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.h b/src/Databases/Iceberg/DatabaseIcebergSettings.h index 5d9d120efed..4e5bc0defba 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.h +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.h @@ -16,6 +16,7 @@ class SettingsChanges; #define DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(CLASS_NAME, M) \ M(CLASS_NAME, String) \ M(CLASS_NAME, UInt64) \ + M(CLASS_NAME, Bool) \ M(CLASS_NAME, DatabaseIcebergCatalogType) \ M(CLASS_NAME, DatabaseIcebergStorageType) diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index db8ad443c0e..97b2bfeeef9 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -68,4 +68,19 @@ const DB::NamesAndTypesList & TableMetadata::getSchema() const return schema; } +void TableMetadata::setStorageCredentials(std::shared_ptr credentials_) +{ + if (!with_storage_credentials) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Storage credentials were not requested"); + + storage_credentials = std::move(credentials_); +} + +std::shared_ptr TableMetadata::getStorageCredentials() const +{ + if (!with_storage_credentials) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + return storage_credentials; +} } diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index aa574509c8a..9657ef6ba41 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -2,6 +2,7 @@ #include #include #include +#include namespace Iceberg { @@ -16,6 +17,7 @@ public: TableMetadata & withLocation() { with_location = true; return *this; } TableMetadata & withSchema() { with_schema = true; return *this; } + TableMetadata & withStorageCredentials() { with_storage_credentials = true; return *this; } void setLocation(const std::string & location_); std::string getLocation(bool path_only) const; @@ -23,8 +25,12 @@ public: void setSchema(const DB::NamesAndTypesList & schema_); const DB::NamesAndTypesList & getSchema() const; + void setStorageCredentials(std::shared_ptr credentials_); + std::shared_ptr getStorageCredentials() const; + bool requiresLocation() const { return with_location; } bool requiresSchema() const { return with_schema; } + bool requiresCredentials() const { return with_storage_credentials; } private: /// Starts with s3://, file://, etc. @@ -34,10 +40,12 @@ private: std::string path; DB::NamesAndTypesList schema; - std::string credentials; + /// Storage credentials, which are called "vended credentials". + std::shared_ptr storage_credentials; bool with_location = false; bool with_schema = false; + bool with_storage_credentials = false; }; @@ -88,4 +96,5 @@ protected: const std::string warehouse; }; + } diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 149a26f107a..e5676265855 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -170,7 +171,7 @@ void RestCatalog::parseCatalogConfigurationSettings(const Poco::JSON::Object::Pt result.default_base_location = object->get("default-base-location").extract(); } -DB::HTTPHeaderEntries RestCatalog::getHeaders(bool update_token) const +DB::HTTPHeaderEntries RestCatalog::getAuthHeaders(bool update_token) const { /// Option 1: user specified auth header manually. /// Header has format: 'Authorization: '. @@ -248,7 +249,8 @@ std::optional RestCatalog::getStorageType() const DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( const std::string & endpoint, - const Poco::URI::QueryParameters & params) const + const Poco::URI::QueryParameters & params, + const DB::HTTPHeaderEntries & headers) const { const auto & context = getContext(); @@ -258,13 +260,15 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( auto create_buffer = [&](bool update_token) { - auto headers = getHeaders(update_token); + auto result_headers = getAuthHeaders(update_token); + std::move(headers.begin(), headers.end(), std::back_inserter(result_headers)); + return DB::BuilderRWBufferFromHTTP(url) .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) .withSettings(getContext()->getReadSettings()) .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) .withHostFilter(&getContext()->getRemoteHostFilter()) - .withHeaders(headers) + .withHeaders(result_headers) .withDelayInit(false) .withSkipNotFound(false) .create(credentials); @@ -521,8 +525,12 @@ bool RestCatalog::getTableMetadataImpl( { LOG_TEST(log, "Checking table {} in namespace {}", table_name, namespace_name); + DB::HTTPHeaderEntries headers; + if (result.requiresCredentials()) + headers.emplace_back("X-Iceberg-Access-Delegation", "vended-credentials"); + const auto endpoint = std::string(namespaces_endpoint) + "/" + namespace_name + "/tables/" + table_name; - auto buf = createReadBuffer(config.prefix / endpoint); + auto buf = createReadBuffer(config.prefix / endpoint, /* params */{}, headers); if (buf->eof()) { @@ -533,6 +541,7 @@ bool RestCatalog::getTableMetadataImpl( String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); + /// TODO: remove before merge because it might contain credentials. LOG_TEST(log, "Received metadata for table {}: {}", table_name, json_str); Poco::JSON::Parser parser; @@ -543,9 +552,10 @@ bool RestCatalog::getTableMetadataImpl( if (!metadata_object) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + std::string location; if (result.requiresLocation()) { - const auto location = metadata_object->get("location").extract(); + location = metadata_object->get("location").extract(); result.setLocation(location); LOG_TEST(log, "Location for table {}: {}", table_name, location); } @@ -556,23 +566,37 @@ bool RestCatalog::getTableMetadataImpl( result.setSchema(DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first); } - // if (result.requiresCredentials()) - // { - // try - // { - // const auto credentials_endpoint = std::filesystem::path(endpoint) / "credentials"; - // auto credentials_buf = createReadBuffer(config.prefix / credentials_endpoint); + if (result.requiresCredentials() && object->has("config")) + { + auto config_object = object->get("config").extract(); + if (!config_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse config result"); - // String credentials_json_str; - // readJSONObjectPossiblyInvalid(credentials_json_str, *credentials_buf); + auto storage_type = parseStorageTypeFromLocation(location); + switch (storage_type) + { + case StorageType::S3: + { + static constexpr auto access_key_id_str = "s3.access-key-id"; + static constexpr auto secret_access_key_str = "s3.secret-access-key"; + static constexpr auto session_token_str = "s3.session-token"; - // LOG_TEST(log, "Credentials : {}", credentials_json_str); - // } - // catch (...) - // { - // DB::tryLogCurrentException(log); - // } - // } + std::string access_key_id, secret_access_key, session_token; + if (config_object->has(access_key_id_str)) + access_key_id = config_object->get(access_key_id_str).extract(); + if (config_object->has(secret_access_key_str)) + secret_access_key = config_object->get(secret_access_key_str).extract(); + if (config_object->has(session_token_str)) + session_token = config_object->get(session_token_str).extract(); + + result.setStorageCredentials( + std::make_shared(access_key_id, secret_access_key, session_token)); + break; + } + default: + break; + } + } return true; } diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 570707e3aa0..4505e020580 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -79,7 +79,8 @@ private: DB::ReadWriteBufferFromHTTPPtr createReadBuffer( const std::string & endpoint, - const Poco::URI::QueryParameters & params = {}) const; + const Poco::URI::QueryParameters & params = {}, + const DB::HTTPHeaderEntries & headers = {}) const; Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; @@ -107,7 +108,7 @@ private: Config loadConfig(); std::string retrieveAccessToken() const; - DB::HTTPHeaderEntries getHeaders(bool update_token = false) const; + DB::HTTPHeaderEntries getAuthHeaders(bool update_token = false) const; static void parseCatalogConfigurationSettings(const Poco::JSON::Object::Ptr & object, Config & result); }; diff --git a/src/Databases/Iceberg/StorageCredentials.h b/src/Databases/Iceberg/StorageCredentials.h new file mode 100644 index 00000000000..0cf0ffea4a5 --- /dev/null +++ b/src/Databases/Iceberg/StorageCredentials.h @@ -0,0 +1,50 @@ +#pragma once +#include +#include +#include + +namespace DB::ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace Iceberg +{ + +class IStorageCredentials +{ +public: + virtual ~IStorageCredentials() = default; + + virtual void addCredentialsToEngineArgs(DB::ASTs & engine_args) const = 0; +}; + +class S3Credentials final : public IStorageCredentials +{ +public: + S3Credentials( + const std::string & access_key_id_, + const std::string & secret_access_key_, + const std::string session_token_) + : access_key_id(access_key_id_) + , secret_access_key(secret_access_key_) + , session_token(session_token_) + {} + + void addCredentialsToEngineArgs(DB::ASTs & engine_args) const override + { + if (engine_args.size() != 1) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Storage credentials specified in AST already"); + + engine_args.push_back(std::make_shared(access_key_id)); + engine_args.push_back(std::make_shared(secret_access_key)); + engine_args.push_back(std::make_shared(session_token)); + } + +private: + std::string access_key_id; + std::string secret_access_key; + std::string session_token; +}; + +} From bec1e4af97b15bdfa55f65f1ba175b8d1ec30264 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 19:03:22 +0100 Subject: [PATCH 25/39] Add a comment --- src/Databases/Iceberg/RestCatalog.cpp | 8 ++++++++ src/Databases/Iceberg/StorageCredentials.h | 1 + 2 files changed, 9 insertions(+) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index e5676265855..9509f40b3d6 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -527,7 +527,15 @@ bool RestCatalog::getTableMetadataImpl( DB::HTTPHeaderEntries headers; if (result.requiresCredentials()) + { + /// Header `X-Iceberg-Access-Delegation` tells catalog to include storage credentials in LoadTableResponse. + /// Value can be one of the two: + /// 1. `vended-credentials` + /// 2. `remote-signing` + /// Currently we support only the first. + /// https://github.com/apache/iceberg/blob/3badfe0c1fcf0c0adfc7aa4a10f0b50365c48cf9/open-api/rest-catalog-open-api.yaml#L1832 headers.emplace_back("X-Iceberg-Access-Delegation", "vended-credentials"); + } const auto endpoint = std::string(namespaces_endpoint) + "/" + namespace_name + "/tables/" + table_name; auto buf = createReadBuffer(config.prefix / endpoint, /* params */{}, headers); diff --git a/src/Databases/Iceberg/StorageCredentials.h b/src/Databases/Iceberg/StorageCredentials.h index 0cf0ffea4a5..90c20262cf4 100644 --- a/src/Databases/Iceberg/StorageCredentials.h +++ b/src/Databases/Iceberg/StorageCredentials.h @@ -22,6 +22,7 @@ public: class S3Credentials final : public IStorageCredentials { public: + /// TODO: support region as well. S3Credentials( const std::string & access_key_id_, const std::string & secret_access_key_, From 8239663c0504c38ed7d9ffa5f64187e239a212d1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 19:23:19 +0100 Subject: [PATCH 26/39] Add experimental flag --- src/Core/Settings.cpp | 3 +++ src/Core/SettingsChangesHistory.cpp | 1 + src/Databases/Iceberg/DatabaseIceberg.cpp | 20 ++++++++++++++----- .../integration/test_database_iceberg/test.py | 1 + 4 files changed, 20 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 2526334d290..772585d3f46 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5533,6 +5533,9 @@ Allow to ignore schema evolution in Iceberg table engine and read all data using :::note Enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema. ::: +)", 0) \ + DECLARE(Bool, allow_experimental_database_iceberg, false, R"( +Allow experimental database engine Iceberg )", 0) \ DECLARE(Bool, allow_deprecated_error_prone_window_functions, false, R"( Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 768779c37db..490d21c44a2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -60,6 +60,7 @@ static std::initializer_list +#include #include #include @@ -44,10 +45,15 @@ namespace DatabaseIcebergSetting extern const DatabaseIcebergSettingsString storage_endpoint; extern const DatabaseIcebergSettingsBool vended_credentials; } +namespace Setting +{ + extern const SettingsBool allow_experimental_database_iceberg; +} namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int SUPPORT_IS_DISABLED; } namespace @@ -349,6 +355,14 @@ void registerDatabaseIceberg(DatabaseFactory & factory) { auto create_fn = [](const DatabaseFactory::Arguments & args) { + if (!args.create_query.attach + && !args.context->getSettingsRef()[Setting::allow_experimental_database_iceberg]) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, + "DatabaseIceberg engine is experimental. " + "To allow its usage, enable setting allow_experimental_database_iceberg"); + } + const auto * database_engine_define = args.create_query.storage; const auto & database_engine_name = args.engine_name; @@ -357,13 +371,9 @@ void registerDatabaseIceberg(DatabaseFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); ASTs & engine_args = function_define->arguments->children; - if (engine_args.empty()) + if (engine_args.size() < 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); - const size_t max_args_num = 1; - if (engine_args.size() != max_args_num) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine must have {} arguments", max_args_num); - for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.context); diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index c0f0087522d..2896b87f63e 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -121,6 +121,7 @@ def create_clickhouse_iceberg_database(started_cluster, node, name): node.query( f""" DROP DATABASE IF EXISTS {name}; +SET allow_experimental_database_iceberg=true; CREATE DATABASE {name} ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') SETTINGS catalog_type = 'rest', storage_endpoint = 'http://minio:9000/warehouse', From 8e345cfccebf29a1b477aebd08caf4654b6c6552 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Nov 2024 19:27:46 +0100 Subject: [PATCH 27/39] Fix style check --- src/Databases/Iceberg/DatabaseIceberg.cpp | 9 +-------- src/Databases/Iceberg/RestCatalog.cpp | 1 + 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 3083b40d118..dd559ba8b92 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -24,13 +24,6 @@ #include #include -namespace CurrentMetrics -{ - extern const Metric IcebergCatalogThreads; - extern const Metric IcebergCatalogThreadsActive; - extern const Metric IcebergCatalogThreadsScheduled; -} - namespace DB { @@ -233,7 +226,7 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ throw Exception( ErrorCodes::BAD_ARGUMENTS, "Either vended credentials need to be enabled " - "or storage credentials need to be specified in database engine arguements in CREATE query"); + "or storage credentials need to be specified in database engine arguments in CREATE query"); } LOG_TEST(log, "Using table endpoint: {}", table_endpoint); diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 9509f40b3d6..a3d8a2a2d5a 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -24,6 +24,7 @@ namespace DB::ErrorCodes extern const int ICEBERG_CATALOG_ERROR; extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } namespace Iceberg From ea1827ca5846d69e71af4ebfa080fbc4f5b2df96 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 26 Nov 2024 11:11:41 +0100 Subject: [PATCH 28/39] Fix style check --- src/Databases/enableAllExperimentalSettings.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/enableAllExperimentalSettings.cpp b/src/Databases/enableAllExperimentalSettings.cpp index 1be54664bc9..2b102095632 100644 --- a/src/Databases/enableAllExperimentalSettings.cpp +++ b/src/Databases/enableAllExperimentalSettings.cpp @@ -46,6 +46,7 @@ void enableAllExperimentalSettings(ContextMutablePtr context) context->setSetting("enable_zstd_qat_codec", 1); context->setSetting("allow_create_index_without_type", 1); context->setSetting("allow_experimental_s3queue", 1); + context->setSetting("allow_experimental_database_iceberg", 1); /// clickhouse-private settings context->setSetting("allow_experimental_shared_set_join", 1); From eb9d5705a9055c505cc5aad169118927e4d321d4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 3 Dec 2024 12:22:22 +0100 Subject: [PATCH 29/39] Correct merge --- src/Databases/Iceberg/RestCatalog.cpp | 11 ++++++++++- src/Interpreters/Context.h | 3 +-- .../ObjectStorage/DataLakes/IcebergMetadata.cpp | 2 +- .../ObjectStorage/DataLakes/IcebergMetadata.h | 1 + 4 files changed, 13 insertions(+), 4 deletions(-) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index a3d8a2a2d5a..262ac53e99f 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -27,6 +28,11 @@ namespace DB::ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace DB::Setting +{ + extern const SettingsBool iceberg_engine_ignore_schema_evolution; +} + namespace Iceberg { @@ -571,8 +577,11 @@ bool RestCatalog::getTableMetadataImpl( if (result.requiresSchema()) { + const auto & settings = getContext()->getSettingsRef(); int format_version = metadata_object->getValue("format-version"); - result.setSchema(DB::IcebergMetadata::parseTableSchema(metadata_object, format_version, true).first); + auto schema = DB::IcebergMetadata::parseTableSchema( + metadata_object, log, format_version, settings[DB::Setting::iceberg_engine_ignore_schema_evolution]).first; + result.setSchema(schema); } if (result.requiresCredentials() && object->has("config")) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 6ddb3c1c0f5..9286de76427 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1077,8 +1077,6 @@ public: void clearMarkCache() const; ThreadPool & getLoadMarksThreadpool() const; - ThreadPool & getIcebergCatalogThreadpool() const; - void setPrimaryIndexCache(const String & cache_policy, size_t max_cache_size_in_bytes, double size_ratio); void updatePrimaryIndexCacheConfiguration(const Poco::Util::AbstractConfiguration & config); std::shared_ptr getPrimaryIndexCache() const; @@ -1125,6 +1123,7 @@ public: size_t getPrefetchThreadpoolSize() const; ThreadPool & getBuildVectorSimilarityIndexThreadPool() const; + ThreadPool & getIcebergCatalogThreadpool() const; /// Settings for MergeTree background tasks stored in config.xml BackgroundTaskSchedulingSettings getBackgroundProcessingTaskSchedulingSettings() const; diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index a93af4ee780..7b2ad165972 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -321,7 +321,7 @@ parseTableSchemaV1Method(const Poco::JSON::Object::Ptr & metadata_object, bool i return {schema, current_schema_id}; } -std::pair parseTableSchema( +std::pair IcebergMetadata::parseTableSchema( const Poco::JSON::Object::Ptr & metadata_object, LoggerPtr metadata_logger, int format_version, bool ignore_schema_evolution) { Poco::JSON::Object::Ptr schema; diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index b28de471b40..5f62dbfb2ef 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -98,6 +98,7 @@ public: static std::pair parseTableSchema( const Poco::JSON::Object::Ptr & metadata_object, + LoggerPtr metadata_logger, int format_version, bool ignore_schema_evolution); From c434a3f87af75e131bc0bb6e21967f1640ffd159 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 4 Dec 2024 19:04:43 +0100 Subject: [PATCH 30/39] Support oauth_server_uri --- src/Databases/Iceberg/DatabaseIceberg.cpp | 4 +- .../Iceberg/DatabaseIcebergSettings.cpp | 1 + src/Databases/Iceberg/RestCatalog.cpp | 44 +++++++++++++++---- src/Databases/Iceberg/RestCatalog.h | 2 + 4 files changed, 41 insertions(+), 10 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index dd559ba8b92..8544239d6b1 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -36,6 +36,7 @@ namespace DatabaseIcebergSetting extern const DatabaseIcebergSettingsString auth_header; extern const DatabaseIcebergSettingsString auth_scope; extern const DatabaseIcebergSettingsString storage_endpoint; + extern const DatabaseIcebergSettingsString oauth_server_uri; extern const DatabaseIcebergSettingsBool vended_credentials; } namespace Setting @@ -119,6 +120,7 @@ std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const settings[DatabaseIcebergSetting::catalog_credential].value, settings[DatabaseIcebergSetting::auth_scope].value, settings[DatabaseIcebergSetting::auth_header], + settings[DatabaseIcebergSetting::oauth_server_uri].value, Context::getGlobalContextInstance()); } } @@ -364,7 +366,7 @@ void registerDatabaseIceberg(DatabaseFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); ASTs & engine_args = function_define->arguments->children; - if (engine_args.size() < 1) + if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", database_engine_name); for (auto & engine_arg : engine_args) diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index de04fc0bd11..33374edbb6d 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes DECLARE(String, catalog_credential, "", "", 0) \ DECLARE(Bool, vended_credentials, true, "Use vended credentials (storage credentials) from catalog", 0) \ DECLARE(String, auth_scope, "PRINCIPAL_ROLE:ALL", "Authorization scope for client credentials or token exchange", 0) \ + DECLARE(String, oauth_server_uri, "", "OAuth server uri", 0) \ DECLARE(String, warehouse, "", "Warehouse name inside the catalog", 0) \ DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ DECLARE(String, storage_endpoint, "", "Object storage endpoint", 0) \ diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 262ac53e99f..b138cc1c14b 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -101,6 +101,13 @@ StorageType parseStorageTypeFromLocation(const std::string & location) return *storage_type; } +std::string correctAPIURI(const std::string & uri) +{ + if (uri.ends_with("v1")) + return uri; + return std::filesystem::path(uri) / "v1"; +} + } std::string RestCatalog::Config::toString() const @@ -122,12 +129,14 @@ RestCatalog::RestCatalog( const std::string & catalog_credential_, const std::string & auth_scope_, const std::string & auth_header_, + const std::string & oauth_server_uri_, DB::ContextPtr context_) : ICatalog(warehouse_) , DB::WithContext(context_) - , base_url(base_url_) + , base_url(correctAPIURI(base_url_)) , log(getLogger("RestCatalog(" + warehouse_ + ")")) , auth_scope(auth_scope_) + , oauth_server_uri(oauth_server_uri_) { if (!catalog_credential_.empty()) { @@ -217,14 +226,30 @@ std::string RestCatalog::retrieveAccessToken() const headers.emplace_back("Content-Type", "application/x-www-form-urlencoded"); headers.emplace_back("Accepts", "application/json; charset=UTF-8"); - Poco::URI url(base_url / oauth_tokens_endpoint); - Poco::URI::QueryParameters params = { - {"grant_type", "client_credentials"}, - {"scope", auth_scope}, - {"client_id", client_id}, - {"client_secret", client_secret}, - }; - url.setQueryParameters(params); + Poco::URI url; + DB::ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback; + if (oauth_server_uri.empty()) + { + url = Poco::URI(base_url / oauth_tokens_endpoint); + + Poco::URI::QueryParameters params = { + {"grant_type", "client_credentials"}, + {"scope", auth_scope}, + {"client_id", client_id}, + {"client_secret", client_secret}, + }; + url.setQueryParameters(params); + } + else + { + url = Poco::URI(oauth_server_uri); + out_stream_callback = [&](std::ostream & os) + { + os << fmt::format( + "grant_type=client_credentials&scope={}&client_id={}&client_secret={}", + auth_scope, client_id, client_secret); + }; + } const auto & context = getContext(); auto wb = DB::BuilderRWBufferFromHTTP(url) @@ -233,6 +258,7 @@ std::string RestCatalog::retrieveAccessToken() const .withSettings(context->getReadSettings()) .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) .withHostFilter(&context->getRemoteHostFilter()) + .withOutCallback(std::move(out_stream_callback)) .withSkipNotFound(false) .withHeaders(headers) .create(credentials); diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index 4505e020580..aab8be6ed8d 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -24,6 +24,7 @@ public: const std::string & catalog_credential_, const std::string & auth_scope_, const std::string & auth_header_, + const std::string & oauth_server_uri_, DB::ContextPtr context_); ~RestCatalog() override = default; @@ -73,6 +74,7 @@ private: std::string client_id; std::string client_secret; std::string auth_scope; + std::string oauth_server_uri; mutable std::optional access_token; Poco::Net::HTTPBasicCredentials credentials{}; From cd254a6ee81c15890d7ab90106e860b86d74c5d7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 4 Dec 2024 19:41:32 +0100 Subject: [PATCH 31/39] Remove useless setting --- src/Core/SettingsEnums.cpp | 7 ---- src/Core/SettingsEnums.h | 10 ----- src/Databases/Iceberg/DatabaseIceberg.cpp | 37 ++++++++----------- src/Databases/Iceberg/DatabaseIceberg.h | 7 ++-- .../Iceberg/DatabaseIcebergSettings.cpp | 1 - .../Iceberg/DatabaseIcebergSettings.h | 1 - .../Iceberg/DatabaseIcebergStorageType.h | 14 +++++++ src/Databases/Iceberg/ICatalog.cpp | 32 ++++++++++++++++ src/Databases/Iceberg/ICatalog.h | 4 ++ src/Databases/Iceberg/RestCatalog.cpp | 26 ------------- 10 files changed, 68 insertions(+), 71 deletions(-) create mode 100644 src/Databases/Iceberg/DatabaseIcebergStorageType.h diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 8f7ff6f149c..01b9229ebb7 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -283,11 +283,4 @@ IMPLEMENT_SETTING_ENUM( IMPLEMENT_SETTING_ENUM(DatabaseIcebergCatalogType, ErrorCodes::BAD_ARGUMENTS, {{"rest", DatabaseIcebergCatalogType::REST}}) -IMPLEMENT_SETTING_ENUM(DatabaseIcebergStorageType, ErrorCodes::BAD_ARGUMENTS, - {{"s3", DatabaseIcebergStorageType::S3}, - {"azure", DatabaseIcebergStorageType::Azure}, - {"hdfs", DatabaseIcebergStorageType::HDFS}, - {"local", DatabaseIcebergStorageType::Local}, - }) - } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 33ef8e9623f..aefbc45d411 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -366,14 +366,4 @@ enum class DatabaseIcebergCatalogType : uint8_t DECLARE_SETTING_ENUM(DatabaseIcebergCatalogType) -enum class DatabaseIcebergStorageType : uint8_t -{ - S3, - Azure, - Local, - HDFS, -}; - -DECLARE_SETTING_ENUM(DatabaseIcebergStorageType) - } diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 8544239d6b1..62af8288933 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -30,7 +30,6 @@ namespace DB namespace DatabaseIcebergSetting { extern const DatabaseIcebergSettingsDatabaseIcebergCatalogType catalog_type; - extern const DatabaseIcebergSettingsDatabaseIcebergStorageType storage_type; extern const DatabaseIcebergSettingsString warehouse; extern const DatabaseIcebergSettingsString catalog_credential; extern const DatabaseIcebergSettingsString auth_header; @@ -72,18 +71,17 @@ DatabaseIceberg::DatabaseIceberg( const std::string & database_name_, const std::string & url_, const DatabaseIcebergSettings & settings_, - ASTPtr database_engine_definition_, - ContextPtr context_) + ASTPtr database_engine_definition_) : IDatabase(database_name_) , url(url_) , settings(settings_) , database_engine_definition(database_engine_definition_) , log(getLogger("DatabaseIceberg(" + database_name_ + ")")) { - validateSettings(context_); + validateSettings(); } -void DatabaseIceberg::validateSettings(const ContextPtr & context_) +void DatabaseIceberg::validateSettings() { if (settings[DatabaseIcebergSetting::warehouse].value.empty()) { @@ -91,18 +89,6 @@ void DatabaseIceberg::validateSettings(const ContextPtr & context_) ErrorCodes::BAD_ARGUMENTS, "`warehouse` setting cannot be empty. " "Please specify 'SETTINGS warehouse=' in the CREATE DATABASE query"); } - - if (!settings[DatabaseIcebergSetting::storage_type].changed) - { - auto catalog = getCatalog(context_); - const auto storage_type = catalog->getStorageType(); - if (!storage_type) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Storage type is not found in catalog config. " - "Please specify it manually via 'SETTINGS storage_type=' in CREATE DATABASE query"); - } - } } std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const @@ -127,11 +113,11 @@ std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const return catalog_impl; } -std::shared_ptr DatabaseIceberg::getConfiguration() const +std::shared_ptr DatabaseIceberg::getConfiguration(DatabaseIcebergStorageType type) const { /// TODO: add tests for azure, local storage types. - switch (settings[DatabaseIcebergSetting::storage_type].value) + switch (type) { #if USE_AWS_S3 case DB::DatabaseIcebergStorageType::S3: @@ -234,7 +220,15 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ LOG_TEST(log, "Using table endpoint: {}", table_endpoint); const auto columns = ColumnsDescription(table_metadata.getSchema()); - const auto configuration = getConfiguration(); + + DatabaseIcebergStorageType storage_type; + auto storage_type_from_catalog = catalog->getStorageType(); + if (storage_type_from_catalog.has_value()) + storage_type = storage_type_from_catalog.value(); + else + storage_type = table_metadata.getStorageType(); + + const auto configuration = getConfiguration(storage_type); /// with_table_structure = false: because there will be /// no table structure in table definition AST. @@ -382,8 +376,7 @@ void registerDatabaseIceberg(DatabaseFactory & factory) args.database_name, url, database_settings, - database_engine_define->clone(), - args.context); + database_engine_define->clone()); }; factory.registerDatabase("Iceberg", create_fn, { .supports_arguments = true, .supports_settings = true }); } diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 4b9929a3a76..5138d0f639c 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -18,8 +18,7 @@ public: const std::string & database_name_, const std::string & url_, const DatabaseIcebergSettings & settings_, - ASTPtr database_engine_definition_, - ContextPtr context_); + ASTPtr database_engine_definition_); String getEngineName() const override { return "Iceberg"; } @@ -57,9 +56,9 @@ private: mutable std::shared_ptr catalog_impl; - void validateSettings(const ContextPtr & context_); + void validateSettings(); std::shared_ptr getCatalog(ContextPtr context_) const; - std::shared_ptr getConfiguration() const; + std::shared_ptr getConfiguration(DatabaseIcebergStorageType type) const; std::string getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const; }; diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp index 33374edbb6d..37b4909106b 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.cpp +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.cpp @@ -16,7 +16,6 @@ namespace ErrorCodes #define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE, ALIAS) \ DECLARE(DatabaseIcebergCatalogType, catalog_type, DatabaseIcebergCatalogType::REST, "Catalog type", 0) \ - DECLARE(DatabaseIcebergStorageType, storage_type, DatabaseIcebergStorageType::S3, "Storage type: S3, Local, Azure, HDFS", 0) \ DECLARE(String, catalog_credential, "", "", 0) \ DECLARE(Bool, vended_credentials, true, "Use vended credentials (storage credentials) from catalog", 0) \ DECLARE(String, auth_scope, "PRINCIPAL_ROLE:ALL", "Authorization scope for client credentials or token exchange", 0) \ diff --git a/src/Databases/Iceberg/DatabaseIcebergSettings.h b/src/Databases/Iceberg/DatabaseIcebergSettings.h index 4e5bc0defba..041a99c6d83 100644 --- a/src/Databases/Iceberg/DatabaseIcebergSettings.h +++ b/src/Databases/Iceberg/DatabaseIcebergSettings.h @@ -18,7 +18,6 @@ class SettingsChanges; M(CLASS_NAME, UInt64) \ M(CLASS_NAME, Bool) \ M(CLASS_NAME, DatabaseIcebergCatalogType) \ - M(CLASS_NAME, DatabaseIcebergStorageType) DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseIcebergSettings, DECLARE_SETTING_TRAIT) diff --git a/src/Databases/Iceberg/DatabaseIcebergStorageType.h b/src/Databases/Iceberg/DatabaseIcebergStorageType.h new file mode 100644 index 00000000000..cc3c8f8cb1d --- /dev/null +++ b/src/Databases/Iceberg/DatabaseIcebergStorageType.h @@ -0,0 +1,14 @@ +#include + +namespace DB +{ + +enum class DatabaseIcebergStorageType : uint8_t +{ + S3, + Azure, + Local, + HDFS, +}; + +} diff --git a/src/Databases/Iceberg/ICatalog.cpp b/src/Databases/Iceberg/ICatalog.cpp index 97b2bfeeef9..4568cf95ac0 100644 --- a/src/Databases/Iceberg/ICatalog.cpp +++ b/src/Databases/Iceberg/ICatalog.cpp @@ -12,6 +12,32 @@ namespace DB::ErrorCodes namespace Iceberg { +StorageType parseStorageTypeFromLocation(const std::string & location) +{ + /// Table location in catalog metadata always starts with one of s3://, file://, etc. + /// So just extract this part of the path and deduce storage type from it. + + auto pos = location.find("://"); + if (pos == std::string::npos) + { + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, + "Unexpected path format: {}", location); + } + + auto storage_type_str = location.substr(0, pos); + auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); + + if (!storage_type) + { + throw DB::Exception( + DB::ErrorCodes::NOT_IMPLEMENTED, + "Unsupported storage type: {}", storage_type_str); + } + + return *storage_type; +} + void TableMetadata::setLocation(const std::string & location_) { if (!with_location) @@ -83,4 +109,10 @@ std::shared_ptr TableMetadata::getStorageCredentials() cons return storage_credentials; } + +StorageType TableMetadata::getStorageType() const +{ + return parseStorageTypeFromLocation(location_without_path); +} + } diff --git a/src/Databases/Iceberg/ICatalog.h b/src/Databases/Iceberg/ICatalog.h index 9657ef6ba41..45d9f056477 100644 --- a/src/Databases/Iceberg/ICatalog.h +++ b/src/Databases/Iceberg/ICatalog.h @@ -3,10 +3,12 @@ #include #include #include +#include namespace Iceberg { using StorageType = DB::DatabaseIcebergStorageType; +StorageType parseStorageTypeFromLocation(const std::string & location); /// A class representing table metadata, /// which was received from Catalog. @@ -32,6 +34,8 @@ public: bool requiresSchema() const { return with_schema; } bool requiresCredentials() const { return with_storage_credentials; } + StorageType getStorageType() const; + private: /// Starts with s3://, file://, etc. /// For example, `s3://bucket/` diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index b138cc1c14b..460fce78696 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -75,32 +75,6 @@ DB::HTTPHeaderEntry parseAuthHeader(const std::string & auth_header) return DB::HTTPHeaderEntry(auth_header.substr(0, pos), auth_header.substr(pos + 1)); } -StorageType parseStorageTypeFromLocation(const std::string & location) -{ - /// Table location in catalog metadata always starts with one of s3://, file://, etc. - /// So just extract this part of the path and deduce storage type from it. - - auto pos = location.find("://"); - if (pos == std::string::npos) - { - throw DB::Exception( - DB::ErrorCodes::NOT_IMPLEMENTED, - "Unexpected path format: {}", location); - } - - auto storage_type_str = location.substr(0, pos); - auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); - - if (!storage_type) - { - throw DB::Exception( - DB::ErrorCodes::NOT_IMPLEMENTED, - "Unsupported storage type: {}", storage_type_str); - } - - return *storage_type; -} - std::string correctAPIURI(const std::string & uri) { if (uri.ends_with("v1")) From 9256d370c8370477d9dbe7683c0e2cf84f3a6b82 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Dec 2024 12:48:12 +0100 Subject: [PATCH 32/39] Resolve conflicts --- src/Core/Settings.cpp | 7 ------- src/Databases/Iceberg/DatabaseIceberg.cpp | 3 ++- src/Databases/Iceberg/DatabaseIcebergStorageType.h | 1 + src/Databases/Iceberg/RestCatalog.cpp | 10 +++++----- .../ObjectStorage/DataLakes/IcebergMetadata.cpp | 4 ---- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h | 7 ++----- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 10 ++++++---- 7 files changed, 16 insertions(+), 26 deletions(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index e4b3a28c60e..284a5572bec 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5560,13 +5560,6 @@ Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries u )", 0) \ DECLARE(Int64, prefer_warmed_unmerged_parts_seconds, 0, R"( Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm. -)", 0) \ - DECLARE(Bool, iceberg_engine_ignore_schema_evolution, false, R"( -Allow to ignore schema evolution in Iceberg table engine and read all data using schema specified by the user on table creation or latest schema parsed from metadata on table creation. - -:::note -Enabling this setting can lead to incorrect result as in case of evolved schema all data files will be read using the same schema. -::: )", 0) \ DECLARE(Bool, allow_experimental_database_iceberg, false, R"( Allow experimental database engine Iceberg diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 62af8288933..7a49d11b19e 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -229,10 +229,11 @@ StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_ storage_type = table_metadata.getStorageType(); const auto configuration = getConfiguration(storage_type); + auto storage_settings = std::make_unique(); /// with_table_structure = false: because there will be /// no table structure in table definition AST. - StorageObjectStorage::Configuration::initialize(*configuration, args, context_, /* with_table_structure */false); + StorageObjectStorage::Configuration::initialize(*configuration, args, context_, /* with_table_structure */false, std::move(storage_settings)); return std::make_shared( configuration, diff --git a/src/Databases/Iceberg/DatabaseIcebergStorageType.h b/src/Databases/Iceberg/DatabaseIcebergStorageType.h index cc3c8f8cb1d..9b8dc3a0633 100644 --- a/src/Databases/Iceberg/DatabaseIcebergStorageType.h +++ b/src/Databases/Iceberg/DatabaseIcebergStorageType.h @@ -1,3 +1,4 @@ +#pragma once #include namespace DB diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 460fce78696..ef947e05f7a 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -577,11 +577,11 @@ bool RestCatalog::getTableMetadataImpl( if (result.requiresSchema()) { - const auto & settings = getContext()->getSettingsRef(); - int format_version = metadata_object->getValue("format-version"); - auto schema = DB::IcebergMetadata::parseTableSchema( - metadata_object, log, format_version, settings[DB::Setting::iceberg_engine_ignore_schema_evolution]).first; - result.setSchema(schema); + // int format_version = metadata_object->getValue("format-version"); + auto schema_processor = DB::IcebergSchemaProcessor(); + auto id = DB::IcebergMetadata::parseTableSchema(metadata_object, schema_processor, log); + auto schema = schema_processor.getClickhouseTableSchemaById(id); + result.setSchema(*schema); } if (result.requiresCredentials() && object->has("config")) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 293969d1c7e..8550717d4c7 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -52,9 +52,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Int32 parseTableSchema( - const Poco::JSON::Object::Ptr & metadata_object, IcebergSchemaProcessor & schema_processor, LoggerPtr metadata_logger); - IcebergMetadata::IcebergMetadata( ObjectStoragePtr object_storage_, ConfigurationObserverPtr configuration_, @@ -125,7 +122,6 @@ bool operator!=(const Poco::JSON::Object & first, const Poco::JSON::Object & sec { return !(first == second); } -} DataTypePtr IcebergSchemaProcessor::getSimpleType(const String & type_name) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index ac55d90badc..d4e70030012 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -207,11 +207,8 @@ public: bool supportsExternalMetadataChange() const override { return true; } - static std::pair parseTableSchema( - const Poco::JSON::Object::Ptr & metadata_object, - LoggerPtr metadata_logger, - int format_version, - bool ignore_schema_evolution); + static Int32 parseTableSchema( + const Poco::JSON::Object::Ptr & metadata_object, IcebergSchemaProcessor & schema_processor, LoggerPtr metadata_logger); private: mutable std::unordered_map schema_id_by_data_file; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ba2f9095887..2091e6185b4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -104,10 +104,12 @@ StorageObjectStorage::StorageObjectStorage( try { if (!lazy_init) - if (configuration->hasExternalDynamicMetadata()) - configuration->updateAndGetCurrentSchema(object_storage, context); - else - configuration->update(object_storage, context); + { + if (configuration->hasExternalDynamicMetadata()) + configuration->updateAndGetCurrentSchema(object_storage, context); + else + configuration->update(object_storage, context); + } } catch (...) { From bed4eaaafab943d2b030e142f234f086aaa03626 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Dec 2024 14:04:48 +0100 Subject: [PATCH 33/39] Fix style check --- src/Databases/Iceberg/RestCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index ef947e05f7a..39108b1648e 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -20,12 +20,12 @@ #include #include + namespace DB::ErrorCodes { extern const int ICEBERG_CATALOG_ERROR; extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; - extern const int NOT_IMPLEMENTED; } namespace DB::Setting From 5e51114a6aa87f7c2367bf6f17880b23ab1cb5e9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Dec 2024 18:34:42 +0100 Subject: [PATCH 34/39] Fix bad conflict resolution --- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 8550717d4c7..8aac7b62d48 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -74,6 +74,8 @@ IcebergMetadata::IcebergMetadata( current_schema_id = schema_id; } +namespace +{ enum class ManifestEntryStatus : uint8_t { EXISTING = 0, @@ -123,6 +125,8 @@ bool operator!=(const Poco::JSON::Object & first, const Poco::JSON::Object & sec return !(first == second); } +} + DataTypePtr IcebergSchemaProcessor::getSimpleType(const String & type_name) { From 43e66782124466b5e6f611895dec6ec08c4afb10 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 9 Dec 2024 19:18:10 +0100 Subject: [PATCH 35/39] Review fixes, fix test --- src/Databases/Iceberg/DatabaseIceberg.cpp | 18 +-- src/Databases/Iceberg/DatabaseIceberg.h | 2 +- src/Databases/Iceberg/RestCatalog.cpp | 141 ++++++++++-------- .../DataLakes/IcebergMetadata.cpp | 2 +- .../integration/test_database_iceberg/test.py | 3 +- 5 files changed, 91 insertions(+), 75 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 7a49d11b19e..3818eecb3ae 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -91,7 +91,7 @@ void DatabaseIceberg::validateSettings() } } -std::shared_ptr DatabaseIceberg::getCatalog(ContextPtr) const +std::shared_ptr DatabaseIceberg::getCatalog() const { if (catalog_impl) return catalog_impl; @@ -145,7 +145,7 @@ std::shared_ptr DatabaseIceberg::getConfigu default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Server does not contain support for storage type {}", - settings[DatabaseIcebergSetting::storage_type].value); + type); #endif } } @@ -167,18 +167,18 @@ std::string DatabaseIceberg::getStorageEndpointForTable(const Iceberg::TableMeta bool DatabaseIceberg::empty() const { - return getCatalog(Context::getGlobalContextInstance())->empty(); + return getCatalog()->empty(); } -bool DatabaseIceberg::isTableExist(const String & name, ContextPtr context_) const +bool DatabaseIceberg::isTableExist(const String & name, ContextPtr /* context_ */) const { const auto [namespace_name, table_name] = parseTableName(name); - return getCatalog(context_)->existsTable(namespace_name, table_name); + return getCatalog()->existsTable(namespace_name, table_name); } StoragePtr DatabaseIceberg::tryGetTable(const String & name, ContextPtr context_) const { - auto catalog = getCatalog(context_); + auto catalog = getCatalog(); auto table_metadata = Iceberg::TableMetadata().withLocation().withSchema(); const bool with_vended_credentials = settings[DatabaseIcebergSetting::vended_credentials].value; @@ -256,7 +256,7 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( bool /* skip_not_loaded */) const { Tables tables; - auto catalog = getCatalog(context_); + auto catalog = getCatalog(); const auto iceberg_tables = catalog->getTables(); auto & pool = context_->getIcebergCatalogThreadpool(); @@ -292,10 +292,10 @@ ASTPtr DatabaseIceberg::getCreateDatabaseQuery() const ASTPtr DatabaseIceberg::getCreateTableQueryImpl( const String & name, - ContextPtr context_, + ContextPtr /* context_ */, bool /* throw_on_error */) const { - auto catalog = getCatalog(context_); + auto catalog = getCatalog(); auto table_metadata = Iceberg::TableMetadata().withLocation().withSchema(); const auto [namespace_name, table_name] = parseTableName(name); diff --git a/src/Databases/Iceberg/DatabaseIceberg.h b/src/Databases/Iceberg/DatabaseIceberg.h index 5138d0f639c..94717c522af 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.h +++ b/src/Databases/Iceberg/DatabaseIceberg.h @@ -57,7 +57,7 @@ private: mutable std::shared_ptr catalog_impl; void validateSettings(); - std::shared_ptr getCatalog(ContextPtr context_) const; + std::shared_ptr getCatalog() const; std::shared_ptr getConfiguration(DatabaseIcebergStorageType type) const; std::string getStorageEndpointForTable(const Iceberg::TableMetadata & table_metadata) const; }; diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index 39108b1648e..c901f7b461f 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -36,8 +36,8 @@ namespace DB::Setting namespace Iceberg { -static constexpr auto config_endpoint = "config"; -static constexpr auto namespaces_endpoint = "namespaces"; +static constexpr auto CONFIG_ENDPOINT = "config"; +static constexpr auto NAMESPACES_ENDPOINT = "namespaces"; namespace { @@ -126,7 +126,7 @@ RestCatalog::RestCatalog( RestCatalog::Config RestCatalog::loadConfig() { Poco::URI::QueryParameters params = {{"warehouse", warehouse}}; - auto buf = createReadBuffer(config_endpoint, params); + auto buf = createReadBuffer(CONFIG_ENDPOINT, params); std::string json_str; readJSONObjectPossiblyInvalid(json_str, *buf); @@ -301,26 +301,19 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( bool RestCatalog::empty() const { - try + /// TODO: add a test with empty namespaces and zero namespaces. + bool found_table = false; + auto stop_condition = [&](const std::string & namespace_name) -> bool { - bool found_table = false; - auto stop_condition = [&](const std::string & namespace_name) -> bool - { - const auto tables = getTables(namespace_name, /* limit */1); - found_table = !tables.empty(); - return found_table; - }; - - Namespaces namespaces; - getNamespacesRecursive("", namespaces, stop_condition, /* execute_func */{}); - + const auto tables = getTables(namespace_name, /* limit */1); + found_table = !tables.empty(); return found_table; - } - catch (...) - { - DB::tryLogCurrentException(log); - return true; - } + }; + + Namespaces namespaces; + getNamespacesRecursive("", namespaces, stop_condition, /* execute_func */{}); + + return found_table; } DB::Names RestCatalog::getTables() const @@ -359,6 +352,8 @@ void RestCatalog::getNamespacesRecursive( StopCondition stop_condition, ExecuteFunc func) const { + checkStackSize(); + auto namespaces = getNamespaces(base_namespace); result.reserve(result.size() + namespaces.size()); result.insert(result.end(), namespaces.begin(), namespaces.end()); @@ -384,6 +379,8 @@ Poco::URI::QueryParameters RestCatalog::createParentNamespaceParams(const std::s std::string parent_param; for (const auto & part : parts) { + /// 0x1F is a unit separator + /// https://github.com/apache/iceberg/blob/70d87f1750627b14b3b25a0216a97db86a786992/open-api/rest-catalog-open-api.yaml#L264 if (!parent_param.empty()) parent_param += static_cast(0x1F); parent_param += part; @@ -399,7 +396,7 @@ RestCatalog::Namespaces RestCatalog::getNamespaces(const std::string & base_name try { - auto buf = createReadBuffer(config.prefix / namespaces_endpoint, params); + auto buf = createReadBuffer(config.prefix / NAMESPACES_ENDPOINT, params); auto namespaces = parseNamespaces(*buf, base_namespace); LOG_TEST(log, "Loaded {} namespaces in base namespace {}", namespaces.size(), base_namespace); return namespaces; @@ -431,36 +428,44 @@ RestCatalog::Namespaces RestCatalog::parseNamespaces(DB::ReadBuffer & buf, const LOG_TEST(log, "Received response: {}", json_str); - Poco::JSON::Parser parser; - Poco::Dynamic::Var json = parser.parse(json_str); - const Poco::JSON::Object::Ptr & object = json.extract(); - - auto namespaces_object = object->get("namespaces").extract(); - if (!namespaces_object) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); - - Namespaces namespaces; - for (size_t i = 0; i < namespaces_object->size(); ++i) + try { - auto current_namespace_array = namespaces_object->get(static_cast(i)).extract(); - if (current_namespace_array->size() == 0) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expected namespace array to be non-empty"); + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); - const int idx = static_cast(current_namespace_array->size()) - 1; - const auto current_namespace = current_namespace_array->get(idx).extract(); - const auto full_namespace = base_namespace.empty() - ? current_namespace - : base_namespace + "." + current_namespace; + auto namespaces_object = object->get("namespaces").extract(); + if (!namespaces_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); - namespaces.push_back(full_namespace); + Namespaces namespaces; + for (size_t i = 0; i < namespaces_object->size(); ++i) + { + auto current_namespace_array = namespaces_object->get(static_cast(i)).extract(); + if (current_namespace_array->size() == 0) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expected namespace array to be non-empty"); + + const int idx = static_cast(current_namespace_array->size()) - 1; + const auto current_namespace = current_namespace_array->get(idx).extract(); + const auto full_namespace = base_namespace.empty() + ? current_namespace + : base_namespace + "." + current_namespace; + + namespaces.push_back(full_namespace); + } + + return namespaces; + } + catch (DB::Exception & e) + { + e.addMessage("while parsing JSON: " + json_str); + throw; } - - return namespaces; } DB::Names RestCatalog::getTables(const std::string & base_namespace, size_t limit) const { - const auto endpoint = std::string(namespaces_endpoint) + "/" + base_namespace + "/tables"; + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / base_namespace / "tables"; auto buf = createReadBuffer(config.prefix / endpoint); return parseTables(*buf, base_namespace, limit); } @@ -473,25 +478,34 @@ DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, const std::string & bas String json_str; readJSONObjectPossiblyInvalid(json_str, buf); - Poco::JSON::Parser parser; - Poco::Dynamic::Var json = parser.parse(json_str); - const Poco::JSON::Object::Ptr & object = json.extract(); - - auto identifiers_object = object->get("identifiers").extract(); - if (!identifiers_object) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); - - DB::Names tables; - for (size_t i = 0; i < identifiers_object->size(); ++i) + try { - const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); - const auto table_name = current_table_json->get("name").extract(); + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); - tables.push_back(base_namespace + "." + table_name); - if (limit && tables.size() >= limit) - break; + auto identifiers_object = object->get("identifiers").extract(); + if (!identifiers_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + + DB::Names tables; + for (size_t i = 0; i < identifiers_object->size(); ++i) + { + const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); + const auto table_name = current_table_json->get("name").extract(); + + tables.push_back(base_namespace + "." + table_name); + if (limit && tables.size() >= limit) + break; + } + + return tables; + } + catch (DB::Exception & e) + { + e.addMessage("while parsing JSON: " + json_str); + throw; } - return tables; } bool RestCatalog::existsTable(const std::string & namespace_name, const std::string & table_name) const @@ -544,7 +558,7 @@ bool RestCatalog::getTableMetadataImpl( headers.emplace_back("X-Iceberg-Access-Delegation", "vended-credentials"); } - const auto endpoint = std::string(namespaces_endpoint) + "/" + namespace_name + "/tables/" + table_name; + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / namespace_name / "tables" / table_name; auto buf = createReadBuffer(config.prefix / endpoint, /* params */{}, headers); if (buf->eof()) @@ -556,8 +570,11 @@ bool RestCatalog::getTableMetadataImpl( String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); - /// TODO: remove before merge because it might contain credentials. +#ifdef DEBUG_OR_SANITIZER_BUILD + /// This log message might contain credentials, + /// so log it only for debugging. LOG_TEST(log, "Received metadata for table {}: {}", table_name, json_str); +#endif Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(json_str); diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 8aac7b62d48..2f395a1b59e 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -478,7 +478,7 @@ void IcebergSchemaProcessor::addIcebergTableSchema(Poco::JSON::Object::Ptr schem if (iceberg_table_schemas_by_ids.contains(schema_id)) { chassert(clickhouse_table_schemas_by_ids.contains(schema_id)); - chassert(*iceberg_table_schemas_by_ids.at(schema_id) == *schema_ptr); + // chassert(*iceberg_table_schemas_by_ids.at(schema_id) == *schema_ptr); } else { diff --git a/tests/integration/test_database_iceberg/test.py b/tests/integration/test_database_iceberg/test.py index 2896b87f63e..6dbaccce648 100644 --- a/tests/integration/test_database_iceberg/test.py +++ b/tests/integration/test_database_iceberg/test.py @@ -125,8 +125,7 @@ SET allow_experimental_database_iceberg=true; CREATE DATABASE {name} ENGINE = Iceberg('{BASE_URL}', 'minio', 'minio123') SETTINGS catalog_type = 'rest', storage_endpoint = 'http://minio:9000/warehouse', - warehouse='demo', - storage_type='s3' + warehouse='demo' """ ) From 83ea4857ea3d8ee37756d75a4d0c5611951f8fa0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Dec 2024 19:26:10 +0100 Subject: [PATCH 36/39] Remove redundant --- src/Core/ServerSettings.cpp | 1 - src/Core/SettingsChangesHistory.cpp | 1 - src/Databases/Iceberg/DatabaseIceberg.cpp | 4 ++-- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Core/ServerSettings.cpp b/src/Core/ServerSettings.cpp index ef0385c558f..2169f238c18 100644 --- a/src/Core/ServerSettings.cpp +++ b/src/Core/ServerSettings.cpp @@ -212,7 +212,6 @@ namespace DB DECLARE(UInt64, threadpool_writer_queue_size, 1000000, "Number of tasks which is possible to push into background pool for write requests to object storages", 0) \ DECLARE(UInt64, iceberg_catalog_threadpool_pool_size, 50, "Size of background pool for iceberg catalog", 0) \ DECLARE(UInt64, iceberg_catalog_threadpool_queue_size, 1000000, "Number of tasks which is possible to push into iceberg catalog pool", 0) \ - DECLARE(UInt32, allowed_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ DECLARE(UInt32, allow_feature_tier, 0, "0 - All feature tiers allowed (experimental, beta, production). 1 - Only beta and production feature tiers allowed. 2 - Only production feature tier allowed", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index d6378ce8c62..e59a0046407 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -62,7 +62,6 @@ static std::initializer_list(table_endpoint); - /// We either fetch storage credentials from catalog " - /// "or get storage credentials from database engine arguments + /// We either fetch storage credentials from catalog + /// or get storage credentials from database engine arguments /// in CREATE query (e.g. in `args`). /// Vended credentials can be disabled in catalog itself, /// so we have a separate setting to know whether we should even try to fetch them. From 4737618960b3f3e803ceb0d6d074330065ebce90 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Dec 2024 12:20:56 +0100 Subject: [PATCH 37/39] Fix --- src/Databases/Iceberg/DatabaseIceberg.cpp | 16 +++------------- src/Databases/Iceberg/RestCatalog.cpp | 5 +++-- 2 files changed, 6 insertions(+), 15 deletions(-) diff --git a/src/Databases/Iceberg/DatabaseIceberg.cpp b/src/Databases/Iceberg/DatabaseIceberg.cpp index 3799dbee352..58f4d37baac 100644 --- a/src/Databases/Iceberg/DatabaseIceberg.cpp +++ b/src/Databases/Iceberg/DatabaseIceberg.cpp @@ -259,26 +259,16 @@ DatabaseTablesIteratorPtr DatabaseIceberg::getTablesIterator( auto catalog = getCatalog(); const auto iceberg_tables = catalog->getTables(); - auto & pool = context_->getIcebergCatalogThreadpool(); - DB::ThreadPoolCallbackRunnerLocal runner(pool, "RestCatalog"); - std::mutex mutexx; - for (const auto & table_name : iceberg_tables) { if (filter_by_table_name && !filter_by_table_name(table_name)) continue; - runner([=, &tables, &mutexx, this]{ - auto storage = tryGetTable(table_name, context_); - { - std::lock_guard lock(mutexx); - [[maybe_unused]] bool inserted = tables.emplace(table_name, storage).second; - chassert(inserted); - } - }); + auto storage = tryGetTable(table_name, context_); + [[maybe_unused]] bool inserted = tables.emplace(table_name, storage).second; + chassert(inserted); } - runner.waitForAllToFinishAndRethrowFirstError(); return std::make_unique(tables, getDatabaseName()); } diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index c901f7b461f..ab97755426c 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -289,9 +289,10 @@ DB::ReadWriteBufferFromHTTPPtr RestCatalog::createReadBuffer( } catch (const DB::HTTPException & e) { + const auto status = e.getHTTPStatus(); if (update_token_if_expired && - (e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED - || e.code() == Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN)) + (status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED + || status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN)) { return create_buffer(true); } From f55d547e0f29995cbfd7f9228be341256fb18d66 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Dec 2024 19:28:44 +0100 Subject: [PATCH 38/39] Fix fast test build --- src/Databases/Iceberg/RestCatalog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index ab97755426c..cf92c92bac3 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include #include From 11dc8fa9313f49191313a724be3e8d207fcd636e Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 11 Dec 2024 23:23:33 +0100 Subject: [PATCH 39/39] Fix fast test build --- src/Databases/Iceberg/RestCatalog.cpp | 6 ++++++ src/Databases/Iceberg/RestCatalog.h | 5 +++++ 2 files changed, 11 insertions(+) diff --git a/src/Databases/Iceberg/RestCatalog.cpp b/src/Databases/Iceberg/RestCatalog.cpp index cf92c92bac3..ab51bb8a495 100644 --- a/src/Databases/Iceberg/RestCatalog.cpp +++ b/src/Databases/Iceberg/RestCatalog.cpp @@ -1,3 +1,6 @@ +#include "config.h" + +#if USE_AVRO #include #include @@ -6,6 +9,7 @@ #include #include #include +#include #include #include @@ -640,3 +644,5 @@ bool RestCatalog::getTableMetadataImpl( } } + +#endif diff --git a/src/Databases/Iceberg/RestCatalog.h b/src/Databases/Iceberg/RestCatalog.h index aab8be6ed8d..a1dbe268d0b 100644 --- a/src/Databases/Iceberg/RestCatalog.h +++ b/src/Databases/Iceberg/RestCatalog.h @@ -1,4 +1,7 @@ #pragma once +#include "config.h" + +#if USE_AVRO #include #include #include @@ -115,3 +118,5 @@ private: }; } + +#endif