From 9c6a75c1dde1ed60da51fb98a66306d752ce7242 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 6 Nov 2024 17:08:34 +0100 Subject: [PATCH 01/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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/21] 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