From 369d816d6acf8b945d693f09ba304e98fa1b7dcc Mon Sep 17 00:00:00 2001 From: Gimi Liang Date: Tue, 18 Mar 2025 01:09:48 -0700 Subject: [PATCH] Initial Iceberg support --- build.sh | 1 + contrib/aws-cmake/CMakeLists.txt | 2 +- docker/packager/packager | 2 +- src/CMakeLists.txt | 6 + src/Common/ErrorCodes.cpp | 1 + src/Common/Priority.h | 11 + src/Common/SettingsChanges.cpp | 26 + src/Common/SettingsChanges.h | 7 + src/Core/BaseSettingsFwdMacros.h | 7 + src/Core/BaseSettingsFwdMacrosImpl.h | 11 + src/Core/SettingsEnums.cpp | 3 + src/Core/SettingsEnums.h | 7 + src/Core/UUID.h | 23 + .../ApacheIceberg/ApacheIcebergStorageType.h | 16 + .../ApacheIceberg/DatabaseIceberg.cpp | 353 ++++ src/Databases/ApacheIceberg/DatabaseIceberg.h | 72 + .../ApacheIceberg/DatabaseIcebergSettings.cpp | 93 + .../ApacheIceberg/DatabaseIcebergSettings.h | 38 + src/Databases/DatabaseFactory.cpp | 24 +- src/Databases/IDatabase.h | 4 + .../IO/AsynchronousBoundedReadBuffer.cpp | 355 ++++ src/Disks/IO/AsynchronousBoundedReadBuffer.h | 94 + .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 38 +- src/Disks/ObjectStorages/S3/diskSettings.h | 3 +- src/IO/S3/getObjectInfo.cpp | 34 +- src/IO/S3/getObjectInfo.h | 9 +- src/IO/SeekableReadBuffer.cpp | 63 + src/IO/SeekableReadBuffer.h | 31 + src/IO/WriteBufferFromS3.cpp | 40 +- src/IO/WriteBufferFromS3.h | 6 +- .../FilesystemReadPrefetchesLog.cpp | 61 + .../FilesystemReadPrefetchesLog.h | 51 + src/Parsers/ASTDataType.cpp | 67 + src/Parsers/ASTDataType.h | 40 + src/Processors/Formats/IOutputFormat.h | 3 + .../Formats/Impl/CHColumnToArrowColumn.cpp | 9 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 5 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 2 + src/Processors/ProcessorID.h | 5 +- src/Storages/ExternalStream/CMakeLists.txt | 9 + .../ExternalStream/ExternalStreamSettings.h | 8 +- .../ExternalStream/ExternalStreamTypes.h | 1 + .../ExternalStream/Iceberg/Iceberg.cpp | 352 ++++ src/Storages/ExternalStream/Iceberg/Iceberg.h | 71 + .../Iceberg/IcebergS3Configuration.cpp | 58 + .../Iceberg/IcebergS3Configuration.h | 44 + .../ExternalStream/Iceberg/IcebergSink.cpp | 419 +++++ .../ExternalStream/Iceberg/IcebergSink.h | 105 ++ .../ExternalStream/Iceberg/IcebergSource.cpp | 425 +++++ .../ExternalStream/Iceberg/IcebergSource.h | 184 ++ src/Storages/Iceberg/AvroSchemas.h | 25 + src/Storages/Iceberg/ICatalog.cpp | 132 ++ src/Storages/Iceberg/ICatalog.h | 274 +++ src/Storages/Iceberg/IcebergMetadata.cpp | 122 ++ src/Storages/Iceberg/IcebergMetadata.h | 24 + src/Storages/Iceberg/Manifest.h | 1514 +++++++++++++++++ src/Storages/Iceberg/ManifestList.h | 609 +++++++ src/Storages/Iceberg/Requirement.cpp | 33 + src/Storages/Iceberg/Requirement.h | 50 + src/Storages/Iceberg/RestCatalog.cpp | 892 ++++++++++ src/Storages/Iceberg/RestCatalog.h | 148 ++ src/Storages/Iceberg/Schema.cpp | 265 +++ src/Storages/Iceberg/Schema.h | 16 + src/Storages/Iceberg/SchemaProcessor.cpp | 428 +++++ src/Storages/Iceberg/SchemaProcessor.h | 107 ++ src/Storages/Iceberg/Snapshot.h | 40 + src/Storages/Iceberg/StorageCredentials.h | 45 + src/Storages/Iceberg/Update.cpp | 47 + src/Storages/Iceberg/Update.h | 91 + src/Storages/Iceberg/schemas/manifest.json | 131 ++ .../Iceberg/schemas/manifest_entry.json | 310 ++++ .../NamedCollectionConfiguration.cpp | 185 ++ .../NamedCollectionConfiguration.h | 44 + .../NamedCollections/NamedCollectionUtils.cpp | 434 +++++ .../NamedCollections/NamedCollectionUtils.h | 40 + .../NamedCollections/NamedCollections.cpp | 450 +++++ .../NamedCollections/NamedCollections.h | 139 ++ .../NamedCollectionsHelpers.cpp | 107 ++ .../NamedCollectionsHelpers.h | 18 + .../NamedCollections/NamedCollections_fwd.h | 12 + src/Storages/StorageConfiguration.h | 15 + src/Storages/StorageS3.cpp | 105 +- src/Storages/StorageS3.h | 7 +- src/Storages/StorageS3Settings.cpp | 285 +++- src/Storages/StorageS3Settings.h | 66 +- src/TableFunctions/TableFunctionS3.cpp | 7 +- 87 files changed, 10298 insertions(+), 220 deletions(-) create mode 100644 src/Common/Priority.h create mode 100644 src/Core/BaseSettingsFwdMacros.h create mode 100644 src/Core/BaseSettingsFwdMacrosImpl.h create mode 100644 src/Databases/ApacheIceberg/ApacheIcebergStorageType.h create mode 100644 src/Databases/ApacheIceberg/DatabaseIceberg.cpp create mode 100644 src/Databases/ApacheIceberg/DatabaseIceberg.h create mode 100644 src/Databases/ApacheIceberg/DatabaseIcebergSettings.cpp create mode 100644 src/Databases/ApacheIceberg/DatabaseIcebergSettings.h create mode 100644 src/Disks/IO/AsynchronousBoundedReadBuffer.cpp create mode 100644 src/Disks/IO/AsynchronousBoundedReadBuffer.h create mode 100644 src/IO/SeekableReadBuffer.cpp create mode 100644 src/Interpreters/FilesystemReadPrefetchesLog.cpp create mode 100644 src/Interpreters/FilesystemReadPrefetchesLog.h create mode 100644 src/Parsers/ASTDataType.cpp create mode 100644 src/Parsers/ASTDataType.h create mode 100644 src/Storages/ExternalStream/Iceberg/Iceberg.cpp create mode 100644 src/Storages/ExternalStream/Iceberg/Iceberg.h create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.cpp create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.h create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergSink.cpp create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergSink.h create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergSource.cpp create mode 100644 src/Storages/ExternalStream/Iceberg/IcebergSource.h create mode 100644 src/Storages/Iceberg/AvroSchemas.h create mode 100644 src/Storages/Iceberg/ICatalog.cpp create mode 100644 src/Storages/Iceberg/ICatalog.h create mode 100644 src/Storages/Iceberg/IcebergMetadata.cpp create mode 100644 src/Storages/Iceberg/IcebergMetadata.h create mode 100644 src/Storages/Iceberg/Manifest.h create mode 100644 src/Storages/Iceberg/ManifestList.h create mode 100644 src/Storages/Iceberg/Requirement.cpp create mode 100644 src/Storages/Iceberg/Requirement.h create mode 100644 src/Storages/Iceberg/RestCatalog.cpp create mode 100644 src/Storages/Iceberg/RestCatalog.h create mode 100644 src/Storages/Iceberg/Schema.cpp create mode 100644 src/Storages/Iceberg/Schema.h create mode 100644 src/Storages/Iceberg/SchemaProcessor.cpp create mode 100644 src/Storages/Iceberg/SchemaProcessor.h create mode 100644 src/Storages/Iceberg/Snapshot.h create mode 100644 src/Storages/Iceberg/StorageCredentials.h create mode 100644 src/Storages/Iceberg/Update.cpp create mode 100644 src/Storages/Iceberg/Update.h create mode 100644 src/Storages/Iceberg/schemas/manifest.json create mode 100644 src/Storages/Iceberg/schemas/manifest_entry.json create mode 100644 src/Storages/NamedCollections/NamedCollectionConfiguration.cpp create mode 100644 src/Storages/NamedCollections/NamedCollectionConfiguration.h create mode 100644 src/Storages/NamedCollections/NamedCollectionUtils.cpp create mode 100644 src/Storages/NamedCollections/NamedCollectionUtils.h create mode 100644 src/Storages/NamedCollections/NamedCollections.cpp create mode 100644 src/Storages/NamedCollections/NamedCollections.h create mode 100644 src/Storages/NamedCollections/NamedCollectionsHelpers.cpp create mode 100644 src/Storages/NamedCollections/NamedCollectionsHelpers.h create mode 100644 src/Storages/NamedCollections/NamedCollections_fwd.h create mode 100644 src/Storages/StorageConfiguration.h diff --git a/build.sh b/build.sh index 11c5d4c4937..e17f8f15ddf 100755 --- a/build.sh +++ b/build.sh @@ -81,6 +81,7 @@ cmake .. \ -DENABLE_DEBUG_FUNCS=ON \ -DENABLE_URL_FUNCS=ON \ -DENABLE_AVRO=ON \ + -DENABLE_AWS_S3=ON \ -DENABLE_AWS_MSK_IAM=ON \ -DENABLE_CURL=${is_not_darwin} \ -DENABLE_PULSAR=${is_not_darwin} diff --git a/contrib/aws-cmake/CMakeLists.txt b/contrib/aws-cmake/CMakeLists.txt index bd3043afc96..5d735f97182 100644 --- a/contrib/aws-cmake/CMakeLists.txt +++ b/contrib/aws-cmake/CMakeLists.txt @@ -2,7 +2,7 @@ set(ENABLE_AWS_S3_DEFAULT OFF) set(ENABLE_AWS_MSK_IAM_DEFAULT OFF) # proton: added if(ENABLE_LIBRARIES AND (OS_LINUX OR OS_DARWIN) AND TARGET OpenSSL::Crypto) - set(ENABLE_AWS_S3_DEFAULT OFF) # proton: added https://github.com/timeplus-io/proton/issues/918 + set(ENABLE_AWS_S3_DEFAULT ON) set(ENABLE_AWS_MSK_IAM_DEFAULT ON) # proton: added endif() diff --git a/docker/packager/packager b/docker/packager/packager index b1cfc507366..1659c1ca989 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -218,7 +218,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append('-DENABLE_CYRUS_SASL=ON') cmake_flags.append('-DENABLE_KRB5=ON') cmake_flags.append('-DENABLE_BROTLI=ON') - cmake_flags.append('-DENABLE_S3=ON') + cmake_flags.append('-DENABLE_AWS_S3=ON') cmake_flags.append('-DENABLE_AVRO=ON') cmake_flags.append('-DENABLE_AWS_MSK_IAM=ON') diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 039e58d5e17..37aae58486a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -78,6 +78,9 @@ list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_dele add_headers_and_sources(dbms Disks/IO) add_headers_and_sources(dbms Disks/ObjectStorages) + +add_headers_and_sources(dbms Storages/NamedCollections) + if (USE_LIBPQXX) add_headers_and_sources(dbms Core/PostgreSQL) endif() @@ -97,6 +100,8 @@ if (TARGET ch_contrib::azure_sdk) add_headers_and_sources(dbms Disks/ObjectStorages/AzureBlobStorage) endif() +add_headers_and_sources(dbms Databases/ApacheIceberg) + add_headers_and_sources(dbms Disks/ObjectStorages/Cached) add_headers_and_sources(dbms Disks/ObjectStorages/Web) @@ -208,6 +213,7 @@ add_object_library(clickhouse_server_http Server/HTTP) add_object_library(clickhouse_formats Formats) add_object_library(clickhouse_processors Processors) # proton : starts +add_object_library(clickhouse_storages_iceberg Storages/Iceberg) add_object_library(clickhouse_processors_streaming Processors/Streaming) add_object_library(clickhouse_formats_avro Formats/Avro) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 2b4d165dcf3..25c52ec4b1f 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -626,6 +626,7 @@ M(676, CANNOT_PARSE_IPV6) \ M(677, THREAD_WAS_CANCELED) \ M(678, IO_URING_INIT_FAILED) \ + M(736, ICEBERG_CATALOG_ERROR) \ M(679, IO_URING_SUBMIT_ERROR) \ M(997, UNSUPPORTED) \ M(998, INVALID_INTEGER_STRING) \ diff --git a/src/Common/Priority.h b/src/Common/Priority.h new file mode 100644 index 00000000000..8952fe4dd5a --- /dev/null +++ b/src/Common/Priority.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +/// Common type for priority values. +/// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize +struct Priority +{ + Int64 value = 0; /// Note that lower value means higher priority. + constexpr operator Int64() const { return value; } /// NOLINT +}; diff --git a/src/Common/SettingsChanges.cpp b/src/Common/SettingsChanges.cpp index 9fb4f361e09..7790c272606 100644 --- a/src/Common/SettingsChanges.cpp +++ b/src/Common/SettingsChanges.cpp @@ -46,4 +46,30 @@ Field * SettingsChanges::tryGet(std::string_view name) return &change->value; } +bool SettingsChanges::insertSetting(std::string_view name, const Field & value) +{ + auto it = std::find_if(begin(), end(), [&name](const SettingChange & change) { return change.name == name; }); + if (it != end()) + return false; + emplace_back(name, value); + return true; +} + +void SettingsChanges::setSetting(std::string_view name, const Field & value) +{ + if (auto * setting_value = tryGet(name)) + *setting_value = value; + else + insertSetting(name, value); +} + +bool SettingsChanges::removeSetting(std::string_view name) +{ + auto it = std::find_if(begin(), end(), [&name](const SettingChange & change) { return change.name == name; }); + if (it == end()) + return false; + erase(it); + return true; +} + } diff --git a/src/Common/SettingsChanges.h b/src/Common/SettingsChanges.h index 851dd12751c..83db23016ad 100644 --- a/src/Common/SettingsChanges.h +++ b/src/Common/SettingsChanges.h @@ -30,6 +30,13 @@ class SettingsChanges : public std::vector bool tryGet(std::string_view name, Field & out_value) const; const Field * tryGet(std::string_view name) const; Field * tryGet(std::string_view name); + + /// Inserts element if doesn't exists and returns true, otherwise just returns false + bool insertSetting(std::string_view name, const Field & value); + /// Sets element to value, inserts if doesn't exist + void setSetting(std::string_view name, const Field & value); + /// If element exists - removes it and returns true, otherwise returns false + bool removeSetting(std::string_view name); }; } diff --git a/src/Core/BaseSettingsFwdMacros.h b/src/Core/BaseSettingsFwdMacros.h new file mode 100644 index 00000000000..09a4397e483 --- /dev/null +++ b/src/Core/BaseSettingsFwdMacros.h @@ -0,0 +1,7 @@ +#pragma once + +#define DECLARE_SETTING_TRAIT(CLASS_NAME, TYPE) using CLASS_NAME##TYPE = SettingField##TYPE CLASS_NAME##Impl::*; + +#define DECLARE_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \ + const SettingField##TYPE & operator[](CLASS_NAME##TYPE t) const; \ + SettingField##TYPE & operator[](CLASS_NAME##TYPE t); diff --git a/src/Core/BaseSettingsFwdMacrosImpl.h b/src/Core/BaseSettingsFwdMacrosImpl.h new file mode 100644 index 00000000000..31c6b5dca1f --- /dev/null +++ b/src/Core/BaseSettingsFwdMacrosImpl.h @@ -0,0 +1,11 @@ +#pragma once + +#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \ + const SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) const \ + { \ + return impl.get()->*t; \ + } \ + SettingField##TYPE & CLASS_NAME::operator[](CLASS_NAME##TYPE t) \ + { \ + return impl.get()->*t; \ + } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index a0ab9ed2d27..947d9454017 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -139,4 +139,7 @@ IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, {"JSON", FormatSettings::EscapingRule::JSON}, {"XML", FormatSettings::EscapingRule::XML}, {"Raw", FormatSettings::EscapingRule::Raw}}) + +IMPLEMENT_SETTING_ENUM(IcebergCatalogType, ErrorCodes::BAD_ARGUMENTS, + {{"rest", IcebergCatalogType::REST}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index bbed96827ff..c24beab70d8 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -168,4 +168,11 @@ DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparin DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) +enum class IcebergCatalogType : uint8_t +{ + REST, +}; + +DECLARE_SETTING_ENUM(IcebergCatalogType) + } diff --git a/src/Core/UUID.h b/src/Core/UUID.h index a24dcebdc9e..2a64c57a37e 100644 --- a/src/Core/UUID.h +++ b/src/Core/UUID.h @@ -11,6 +11,29 @@ namespace UUIDHelpers /// Generate random UUID. UUID generateV4(); + constexpr size_t HighBytes = (std::endian::native == std::endian::little) ? 0 : 1; + constexpr size_t LowBytes = (std::endian::native == std::endian::little) ? 1 : 0; + + inline uint64_t getHighBytes(const UUID & uuid) + { + return uuid.toUnderType().items[HighBytes]; + } + + inline uint64_t & getHighBytes(UUID & uuid) + { + return uuid.toUnderType().items[HighBytes]; + } + + inline uint64_t getLowBytes(const UUID & uuid) + { + return uuid.toUnderType().items[LowBytes]; + } + + inline uint64_t & getLowBytes(UUID & uuid) + { + return uuid.toUnderType().items[LowBytes]; + } + const UUID Nil{}; } diff --git a/src/Databases/ApacheIceberg/ApacheIcebergStorageType.h b/src/Databases/ApacheIceberg/ApacheIcebergStorageType.h new file mode 100644 index 00000000000..2e70e0ce749 --- /dev/null +++ b/src/Databases/ApacheIceberg/ApacheIcebergStorageType.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +namespace DB +{ + +enum class ApacheIcebergStorageType : uint8_t +{ + S3, + Azure, + Local, + HDFS, +}; + +} diff --git a/src/Databases/ApacheIceberg/DatabaseIceberg.cpp b/src/Databases/ApacheIceberg/DatabaseIceberg.cpp new file mode 100644 index 00000000000..289bbce68fb --- /dev/null +++ b/src/Databases/ApacheIceberg/DatabaseIceberg.cpp @@ -0,0 +1,353 @@ +#include + +#if USE_AVRO +#include +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace DatabaseApacheIcebergSetting +{ +extern const DatabaseApacheIcebergSettingsIcebergCatalogType catalog_type; +extern const DatabaseApacheIcebergSettingsString warehouse; +extern const DatabaseApacheIcebergSettingsString catalog_credential; +extern const DatabaseApacheIcebergSettingsString auth_header; +extern const DatabaseApacheIcebergSettingsString auth_scope; +extern const DatabaseApacheIcebergSettingsString storage_endpoint; +extern const DatabaseApacheIcebergSettingsString oauth_server_uri; +extern const DatabaseApacheIcebergSettingsBool vended_credentials; +extern const DatabaseApacheIcebergSettingsString rest_catalog_uri; +extern const DatabaseApacheIcebergSettingsBool rest_catalog_sigv4_enabled; +extern const DatabaseApacheIcebergSettingsString rest_catalog_signing_region; +extern const DatabaseApacheIcebergSettingsString rest_catalog_signing_name; +} + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int INVALID_SETTING_VALUE; +extern const int SUPPORT_IS_DISABLED; +extern const int UNKNOWN_DATABASE; +extern const int UNSUPPORTED; +} + +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('.'); + if (pos == std::string::npos) + return {"", name}; + + auto table_name = name.substr(pos + 1); + auto namespace_name = name.substr(0, name.size() - table_name.size() - 1); + return {std::move(namespace_name), std::move(table_name)}; +} + +auto & tableCache() +{ + static LRUCache table_cache{/*max_size_=*/1000}; + return table_cache; +} +} + +DatabaseApacheIceberg::DatabaseApacheIceberg( + const std::string & database_name_, + const std::string & url_, + const DatabaseApacheIcebergSettings & settings_, + ASTPtr database_engine_definition_) + : IDatabase(database_name_) + , url(url_) + , settings(settings_) + , database_engine_definition(database_engine_definition_) + , log(&Poco::Logger::get(fmt::format("DatabaseIceberg({})", database_name))) +{ + validateSettings(); + initCatalog(); + if (!getCatalog()->existsNamespace(getDatabaseName())) + throw DB::Exception(DB::ErrorCodes::UNKNOWN_DATABASE, "Namespace {} does not exist in the catalog", getDatabaseName()); +} + +void DatabaseApacheIceberg::initCatalog() +{ + switch (settings[DatabaseApacheIcebergSetting::catalog_type].value) + { + case DB::IcebergCatalogType::REST: + { + catalog_impl = std::make_shared( + settings[DatabaseApacheIcebergSetting::warehouse].value, + url, + Apache::Iceberg::RestCatalog::Options{ + .catalog_credential = settings[DatabaseApacheIcebergSetting::catalog_credential].value, + .auth_scope = settings[DatabaseApacheIcebergSetting::auth_scope].value, + .auth_header = settings[DatabaseApacheIcebergSetting::auth_header], + .oauth_server_uri = settings[DatabaseApacheIcebergSetting::oauth_server_uri].value, + .enable_sigv4 = settings[DatabaseApacheIcebergSetting::rest_catalog_sigv4_enabled].value, + .signing_region = settings[DatabaseApacheIcebergSetting::rest_catalog_signing_region].value, + .signing_name = settings[DatabaseApacheIcebergSetting::rest_catalog_signing_name].value, + }, + Context::getGlobalContextInstance()); + + break; + } + } +} + +void DatabaseApacheIceberg::validateSettings() +{ + if (settings[DatabaseApacheIcebergSetting::warehouse].value.empty()) + { + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "`warehouse` setting cannot be empty. " + "Please specify 'SETTINGS warehouse=' in the CREATE DATABASE query"); + } +} + +Apache::Iceberg::CatalogPtr DatabaseApacheIceberg::getCatalog() const +{ + return catalog_impl; +} + +std::string DatabaseApacheIceberg::getStorageEndpointForTable(const Apache::Iceberg::TableMetadata & table_metadata) const +{ + auto endpoint_from_settings = settings[DatabaseApacheIcebergSetting::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 DatabaseApacheIceberg::empty() const +{ + return getCatalog()->empty(getDatabaseName()); +} + +bool DatabaseApacheIceberg::isTableExist(const String & name, ContextPtr /* context_ */) const +{ + const auto [namespace_name, table_name] = parseTableName(name); + return getCatalog()->existsTable(namespace_name.empty() ? getDatabaseName() : fmt::format("{}.{}", getDatabaseName(), namespace_name), table_name); +} + +StoragePtr DatabaseApacheIceberg::tryGetTable(const String & name [[maybe_unused]], ContextPtr context_ [[maybe_unused]]) const +{ + auto catalog = getCatalog(); + auto table_metadata = Apache::Iceberg::TableMetadata().withLocation().withSchema(); + + const bool with_vended_credentials = settings[DatabaseApacheIcebergSetting::vended_credentials].value; + if (with_vended_credentials) + table_metadata = table_metadata.withStorageCredentials(); + + auto cache_key = fmt::format("{}.{}", getDatabaseName(), name); + + if (!catalog->tryGetTableMetadata(getDatabaseName(), name, table_metadata)) + { + tableCache().remove(cache_key); + return nullptr; + } + + auto [table, _] = tableCache().getOrSet(cache_key, [&]() { + /// Replace Iceberg Catalog endpoint with storage path endpoint of requested table. + auto table_endpoint = getStorageEndpointForTable(table_metadata); + LOG_TEST(log, "Using table endpoint: {}", table_endpoint); + + auto stream_settings = std::make_shared(); + stream_settings->changes.setSetting("type", "iceberg"); + stream_settings->changes.setSetting("iceberg_storage_endpoint", table_endpoint); + + /// We either fetch storage credentials from catalog + /// or get storage credentials from database engine arguments + /// in CREATE query (e.g. in `args`). + /// Vended credentials can be disabled in catalog itself, + /// so we have a separate setting to know whether we should even try to fetch them. + if (with_vended_credentials) + { + auto storage_credentials = table_metadata.getStorageCredentials(); + if (storage_credentials) + storage_credentials->addCredentialsToSettings(*stream_settings); + } + + ASTStorage stream_storage; + stream_storage.set(stream_storage.settings, stream_settings); + + return StorageExternalStream::create( + /*engine_args=*/ASTs{}, + /*table_id_=*/StorageID{getDatabaseName(), name}, + context_, + ColumnsDescription(table_metadata.getSchema()), + /*comment=*/"", + &stream_storage, + /*attach=*/false); + }); + + return table; +} + +DatabaseTablesIteratorPtr +DatabaseApacheIceberg::getTablesIterator(ContextPtr context_, const FilterByNameFunction & filter_by_table_name) const +{ + Tables tables; + auto catalog = getCatalog(); + const auto iceberg_tables = catalog->getTables(getDatabaseName()); + + 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_); + [[maybe_unused]] bool inserted = tables.emplace(table_name, storage).second; + chassert(inserted); + } + + return std::make_unique(std::move(tables), getDatabaseName()); +} + +ASTPtr DatabaseApacheIceberg::getCreateDatabaseQuery() const +{ + auto create_query = std::make_shared(); + create_query->setDatabase(getDatabaseName()); + create_query->set(create_query->storage, database_engine_definition); + return create_query; +} + +namespace +{ + +void validateCreate(const ASTPtr & query) +{ + auto create = std::dynamic_pointer_cast(query); + if (create->isView()) + throw Exception(ErrorCodes::UNSUPPORTED, "Views are not supported in Iceberg database"); + + if (create->is_dictionary) + throw Exception(ErrorCodes::UNSUPPORTED, "Dictionary is not supported in Iceberg database"); + + if (create->is_external) + { + if (create->storage == nullptr || create->storage->settings == nullptr + || create->storage->settings->changes.tryGet("type") == nullptr + || create->storage->settings->changes.tryGet("type")->get() != "iceberg") + throw Exception(ErrorCodes::UNSUPPORTED, "External streams are not supported in Iceberg database"); + } + + if (create->is_random) + throw Exception(ErrorCodes::UNSUPPORTED, "Random stream is not supported in Iceberg database"); + + if (create->is_virtual) + throw Exception(ErrorCodes::UNSUPPORTED, "Virtual stream is not supported in Iceberg database"); +} + +} + +void DatabaseApacheIceberg::createTable(ContextPtr /*context*/, const String & name, const StoragePtr & table, const ASTPtr & query) +{ + validateCreate(query); + + auto endpoint = settings[DatabaseApacheIcebergSetting::storage_endpoint].value; + auto endpoint_uri = Poco::URI(endpoint); + /// The table location requires a S3 URI (starts with 's3://') + if (endpoint_uri.getScheme().starts_with("http")) + { + std::vector parts; + /// https://Bucket.s3.Region.amazonaws.com + parts.reserve(5); + splitInto<'.'>(parts, endpoint_uri.getHost()); + endpoint = "s3://" + parts[0] + "/"; + } + + getCatalog()->createTable( + getDatabaseName(), + name, + endpoint + (endpoint.ends_with('/') ? "" : "/") + name, + table->getInMemoryMetadata().getColumns().getAllPhysical(), + std::nullopt, + std::nullopt, + /*stage_create=*/false, + {}); +} + +void DatabaseApacheIceberg::dropTable( ContextPtr /*context*/, const String & name, bool /*no_delay*/) +{ + getCatalog()->deleteTable(getDatabaseName(), name); + tableCache().remove(fmt::format("{}.{}", getDatabaseName(), name)); +} + +ASTPtr DatabaseApacheIceberg::getCreateTableQueryImpl(const String & name, ContextPtr /* context_ */, bool /* throw_on_error */) const +{ + auto table_metadata = Apache::Iceberg::TableMetadata().withLocation().withSchema(); + + const auto [namespace_name, table_name] = parseTableName(name); + getCatalog()->getTableMetadata(namespace_name.empty() ? getDatabaseName() : fmt::format("{}.{}", getDatabaseName(), namespace_name), table_name, table_metadata); + + auto create_table_query = std::make_shared(); + + 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); + + 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); + } + + return create_table_query; +} + +bool DatabaseApacheIceberg::configureTableEngine(ASTCreateQuery & create) const +{ + if (create.storage == nullptr) + create.set(create.storage, std::make_shared()); + + create.is_external = true; + if (create.storage->settings == nullptr) + create.storage->set(create.storage->settings, std::make_shared()); + + create.storage->settings->changes.setSetting("type", "iceberg"); + + auto endpoint_from_settings = settings[DatabaseApacheIcebergSetting::storage_endpoint].value; + create.storage->settings->changes.setSetting("iceberg_storage_endpoint", endpoint_from_settings); + + return true; +} + +} + +#endif diff --git a/src/Databases/ApacheIceberg/DatabaseIceberg.h b/src/Databases/ApacheIceberg/DatabaseIceberg.h new file mode 100644 index 00000000000..b7a84a492fb --- /dev/null +++ b/src/Databases/ApacheIceberg/DatabaseIceberg.h @@ -0,0 +1,72 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO +#include +#include +#include +#include + +namespace DB +{ + +class DatabaseApacheIceberg final : public IDatabase, WithContext +{ +public: + explicit DatabaseApacheIceberg( + const std::string & database_name_, + const std::string & url_, + const DatabaseApacheIcebergSettings & settings_, + ASTPtr database_engine_definition_); + + String getEngineName() const override { return "Iceberg"; } + bool configureTableEngine(ASTCreateQuery &) const override; + + 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) const override; + + void shutdown() override { } + + ASTPtr getCreateDatabaseQuery() const override; + + void createTable(ContextPtr /*context*/, const String & /*name*/, const StoragePtr & /*table*/, const ASTPtr & /*query*/) override; + + void dropTable( ContextPtr /*context*/, const String & /*name*/, [[maybe_unused]] bool no_delay) override; + + /// FIXME this should not be a public function, just a temporary trick. + Apache::Iceberg::CatalogPtr getCatalog() const; + +protected: + ASTPtr getCreateTableQueryImpl(const String & table_name, ContextPtr context, bool throw_on_error) const override; + +private: + void validateSettings(); + void initCatalog(); + /*std::shared_ptr getConfiguration(DatabaseIcebergStorageType type) const;*/ + std::string getStorageEndpointForTable(const Apache::Iceberg::TableMetadata & table_metadata) const; + + /// Iceberg Catalog url. + const std::string url; + /// SETTINGS from CREATE query. + const DatabaseApacheIcebergSettings settings; + /// Database engine definition taken from initial CREATE DATABASE query. + const ASTPtr database_engine_definition; + /// Crendetials to authenticate Iceberg Catalog. + Poco::Net::HTTPBasicCredentials credentials; + + mutable Apache::Iceberg::CatalogPtr catalog_impl; + + const Poco::Logger * log; +}; + +} +#endif diff --git a/src/Databases/ApacheIceberg/DatabaseIcebergSettings.cpp b/src/Databases/ApacheIceberg/DatabaseIcebergSettings.cpp new file mode 100644 index 00000000000..1bf9e8667ed --- /dev/null +++ b/src/Databases/ApacheIceberg/DatabaseIcebergSettings.cpp @@ -0,0 +1,93 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNKNOWN_SETTING; +} + +#define DATABASE_ICEBERG_RELATED_SETTINGS(DECLARE) \ + DECLARE(IcebergCatalogType, catalog_type, IcebergCatalogType::REST, "Catalog type", 0) \ + DECLARE(String, catalog_credential, "", "", 0) \ + DECLARE(Bool, vended_credentials, false, "Use vended credentials (storage credentials) from catalog", 0) \ + DECLARE(String, auth_scope, "PRINCIPAL_ROLE:ALL", "Authorization scope for client credentials or token exchange", 0) \ + DECLARE(String, oauth_server_uri, "", "OAuth server uri", 0) \ + DECLARE(String, warehouse, "", "Warehouse name inside the catalog", 0) \ + DECLARE(String, auth_header, "", "Authorization header of format 'Authorization: '", 0) \ + DECLARE(String, storage_endpoint, "", "Storage endpoint", 0) \ + DECLARE(String, rest_catalog_uri, "", "URI identifying the REST catalog server", 0) \ + DECLARE( \ + Bool, rest_catalog_sigv4_enabled, false, "Set to true to sign requests to the REST catalog server using AWS SigV4 protocol", 0) \ + DECLARE(String, rest_catalog_signing_region, "", "The region to use when SigV4 signing a request", 0) \ + DECLARE(String, rest_catalog_signing_name, "", "The service signing name to use when SigV4 signing a request", 0) + +#define LIST_OF_DATABASE_ICEBERG_SETTINGS(M) DATABASE_ICEBERG_RELATED_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(DatabaseApacheIcebergSettingsTraits, LIST_OF_DATABASE_ICEBERG_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(DatabaseApacheIcebergSettingsTraits, LIST_OF_DATABASE_ICEBERG_SETTINGS) + +struct DatabaseApacheIcebergSettingsImpl : public BaseSettings +{ +}; + +#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + DatabaseApacheIcebergSettings##TYPE NAME = &DatabaseApacheIcebergSettingsImpl ::NAME; + +namespace DatabaseApacheIcebergSetting +{ +LIST_OF_DATABASE_ICEBERG_SETTINGS(INITIALIZE_SETTING_EXTERN) +} + +#undef INITIALIZE_SETTING_EXTERN + +DatabaseApacheIcebergSettings::DatabaseApacheIcebergSettings() : impl(std::make_unique()) +{ +} + +DatabaseApacheIcebergSettings::DatabaseApacheIcebergSettings(const DatabaseApacheIcebergSettings & settings) + : impl(std::make_unique(*settings.impl)) +{ +} + +DatabaseApacheIcebergSettings::DatabaseApacheIcebergSettings(DatabaseApacheIcebergSettings && settings) noexcept + : impl(std::make_unique(std::move(*settings.impl))) +{ +} + +DatabaseApacheIcebergSettings::~DatabaseApacheIcebergSettings() = default; + +DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseApacheIcebergSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR) + + +void DatabaseApacheIcebergSettings::applyChanges(const SettingsChanges & changes) +{ + impl->applyChanges(changes); +} + +void DatabaseApacheIcebergSettings::loadFromQuery(const ASTStorage & storage_def) +{ + if (storage_def.settings != nullptr) + { + 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/ApacheIceberg/DatabaseIcebergSettings.h b/src/Databases/ApacheIceberg/DatabaseIcebergSettings.h new file mode 100644 index 00000000000..4a3ede24b76 --- /dev/null +++ b/src/Databases/ApacheIceberg/DatabaseIcebergSettings.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ +class ASTStorage; +struct DatabaseApacheIcebergSettingsImpl; +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, Bool) \ + M(CLASS_NAME, IcebergCatalogType) + +DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseApacheIcebergSettings, DECLARE_SETTING_TRAIT) + +struct DatabaseApacheIcebergSettings +{ + DatabaseApacheIcebergSettings(); + DatabaseApacheIcebergSettings(const DatabaseApacheIcebergSettings & settings); + DatabaseApacheIcebergSettings(DatabaseApacheIcebergSettings && settings) noexcept; + ~DatabaseApacheIcebergSettings(); + + DATABASE_ICEBERG_SETTINGS_SUPPORTED_TYPES(DatabaseApacheIcebergSettings, 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/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 12cc150c53a..9fe591f6596 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include #include #include #include @@ -83,13 +85,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set database_engines{"Ordinary", "Atomic", "Memory", "Dictionary", "Lazy", "Replicated", "MySQL", - "PostgreSQL", "MaterializedPostgreSQL", "SQLite"}; + "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Iceberg"}; /// proton: updated if (!database_engines.contains(engine_name)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Database engine name `{}` does not exist", engine_name); static const std::unordered_set engines_with_arguments{"MySQL", - "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"}; + "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite", "Iceberg"}; /// proton: updated static const std::unordered_set engines_with_table_overrides{"MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); @@ -100,7 +102,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || engine_define->sample_by; - bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL"; + bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL" || engine_name == "Iceberg"; if (has_unexpected_element || (!may_have_settings && engine_define->settings)) throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_AST, @@ -131,6 +133,22 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto cache_expiration_time_seconds = safeGetLiteralValue(arguments[0], "Lazy"); return std::make_shared(database_name, metadata_path, cache_expiration_time_seconds, context); } + else if (engine_name == "Iceberg") + { + ASTs & engine_args = engine_define->engine->arguments->children; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Iceberg database require url argument"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + const auto url = safeGetLiteralValue(engine_args[0], "Iceberg"); + + DatabaseApacheIcebergSettings database_settings; + if (database_engine_define->settings != nullptr) + database_settings.loadFromQuery(*database_engine_define); + return std::make_shared(database_name, url, database_settings, database_engine_define->clone()); + } throw Exception("Unknown database engine: " + engine_name, ErrorCodes::UNKNOWN_DATABASE_ENGINE); } diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 7e9cd9c4ac6..c5b7209b4bd 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -158,6 +158,10 @@ class IDatabase : public std::enable_shared_from_this virtual void startupTables(ThreadPool & /*thread_pool*/, bool /*force_restore*/, bool /*force_attach*/) {} + /// proton: starts + virtual bool configureTableEngine(ASTCreateQuery &) const { return false; } + /// proton: ends + /// Check the existence of the table in memory (attached). virtual bool isTableExist(const String & name, ContextPtr context) const = 0; diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp new file mode 100644 index 00000000000..1371974ec71 --- /dev/null +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -0,0 +1,355 @@ +#include "AsynchronousBoundedReadBuffer.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace CurrentMetrics +{ + extern const Metric AsynchronousReadWait; +} + +namespace ProfileEvents +{ + extern const Event AsynchronousRemoteReadWaitMicroseconds; + extern const Event SynchronousRemoteReadWaitMicroseconds; + extern const Event RemoteFSSeeks; + extern const Event RemoteFSPrefetches; + extern const Event RemoteFSCancelledPrefetches; + extern const Event RemoteFSUnusedPrefetches; + extern const Event RemoteFSPrefetchedReads; + extern const Event RemoteFSUnprefetchedReads; + extern const Event RemoteFSPrefetchedBytes; + extern const Event RemoteFSUnprefetchedBytes; + extern const Event RemoteFSLazySeeks; + extern const Event RemoteFSSeeksWithReset; + extern const Event RemoteFSBuffers; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +static size_t chooseBufferSize(size_t file_size) +{ + /// Buffers used for prefetch or pre-download better to have enough size, but not bigger than the whole file. + return std::min(DBMS_DEFAULT_BUFFER_SIZE, file_size); +} + +AsynchronousBoundedReadBuffer::AsynchronousBoundedReadBuffer( + ImplPtr impl_, + IAsynchronousReader & reader_, + const ReadSettings & settings_, + AsyncReadCountersPtr async_read_counters_, + FilesystemReadPrefetchesLogPtr prefetches_log_) + : ReadBufferFromFileBase(chooseBufferSize(impl_->getFileSize()), nullptr, 0) + , impl(std::move(impl_)) + , read_settings(settings_) + , reader(reader_) + , prefetch_buffer(chooseBufferSize(impl->getFileSize())) + , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") + , current_reader_id(getRandomASCIIString(8)) + , log(&Poco::Logger::get("AsynchronousBoundedReadBuffer")) + , async_read_counters(async_read_counters_) + , prefetches_log(prefetches_log_) +{ + ProfileEvents::increment(ProfileEvents::RemoteFSBuffers); +} + +bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() +{ + if (read_until_position) + { + if (file_offset_of_buffer_end == *read_until_position) /// Everything is already read. + return false; + + if (file_offset_of_buffer_end > *read_until_position) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Read beyond last offset ({} > {}, info: {})", + file_offset_of_buffer_end, *read_until_position, impl->getInfoForLog()); + } + } + + return true; +} + +std::future +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, Priority priority) +{ + IAsynchronousReader::Request request; + request.descriptor = std::make_shared(*impl, async_read_counters); + request.buf = data; + request.size = size; + request.offset = file_offset_of_buffer_end; + request.priority = Priority{static_cast(read_settings.priority + priority.value)}; + request.ignore = bytes_to_ignore; + return reader.submit(request); +} + +void AsynchronousBoundedReadBuffer::prefetch() +{ + if (prefetch_future.valid()) + return; + + if (!hasPendingDataToRead()) + return; + + last_prefetch_info.submit_time = std::chrono::system_clock::now(); + + chassert(prefetch_buffer.size() == chooseBufferSize(impl->getFileSize())); + prefetch_future = asyncReadInto(prefetch_buffer.data(), prefetch_buffer.size(), {}); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetches); +} + +void AsynchronousBoundedReadBuffer::setReadUntilPosition(size_t position) +{ + if (!read_until_position || position != *read_until_position) + { + read_until_position = position; + + /// We must wait on future and reset the prefetch here, because otherwise there might be + /// a race between reading the data in the threadpool and impl->setReadUntilPosition() + /// which reinitializes internal remote read buffer (because if we have a new read range + /// then we need a new range request) and in case of reading from cache we need to request + /// and hold more file segment ranges from cache. + resetPrefetch(FilesystemPrefetchState::CANCELLED_WITH_RANGE_CHANGE); + impl->setReadUntilPosition(*read_until_position); + } +} + +void AsynchronousBoundedReadBuffer::appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch) +{ + FilesystemReadPrefetchesLogElement elem + { + .event_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()), + .query_id = query_id, + .path = impl->getFileName(), + .offset = file_offset_of_buffer_end, + .size = size, + .prefetch_submit_time = last_prefetch_info.submit_time, + .execution_watch = execution_watch ? std::optional(*execution_watch) : std::nullopt, + .priority = last_prefetch_info.priority, + .state = state, + .thread_id = getThreadId(), + .reader_id = current_reader_id, + }; + + if (prefetches_log) + prefetches_log->add(elem); +} + + +bool AsynchronousBoundedReadBuffer::nextImpl() +{ + if (!hasPendingDataToRead()) + return false; + + chassert(file_offset_of_buffer_end <= impl->getFileSize()); + + size_t size, offset; + if (prefetch_future.valid()) + { + ProfileEventTimeIncrement watch(ProfileEvents::AsynchronousRemoteReadWaitMicroseconds); + CurrentMetrics::Increment metric_increment{CurrentMetrics::AsynchronousReadWait}; + + auto result = prefetch_future.get(); + size = result.size; + offset = result.offset; + + prefetch_future = {}; + prefetch_buffer.swap(memory); + + last_prefetch_info = {}; + + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedReads); + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedBytes, size); + } + else + { + ProfileEventTimeIncrement watch(ProfileEvents::SynchronousRemoteReadWaitMicroseconds); + + chassert(memory.size() == chooseBufferSize(impl->getFileSize())); + std::tie(size, offset) = impl->readInto(memory.data(), memory.size(), file_offset_of_buffer_end, bytes_to_ignore); + + ProfileEvents::increment(ProfileEvents::RemoteFSUnprefetchedReads); + ProfileEvents::increment(ProfileEvents::RemoteFSUnprefetchedBytes, size); + } + + bytes_to_ignore = 0; + + chassert(size >= offset); + + size_t bytes_read = size - offset; + if (bytes_read != 0u) + { + /// Adjust the working buffer so that it ignores `offset` bytes. + internal_buffer = Buffer(memory.data(), memory.data() + memory.size()); + working_buffer = Buffer(memory.data() + offset, memory.data() + size); + pos = working_buffer.begin(); + } + + file_offset_of_buffer_end = impl->getFileOffsetOfBufferEnd(); + + /// In case of multiple files for the same file in clickhouse (i.e. log family) + /// file_offset_of_buffer_end will not match getImplementationBufferOffset() + /// so we use [impl->getImplementationBufferOffset(), impl->getFileSize()] + chassert(file_offset_of_buffer_end >= impl->getFileOffsetOfBufferEnd()); + chassert(file_offset_of_buffer_end <= impl->getFileSize()); + + return bytes_read != 0u; +} + + +off_t AsynchronousBoundedReadBuffer::seek(off_t offset, int whence) +{ + ProfileEvents::increment(ProfileEvents::RemoteFSSeeks); + + size_t new_pos; + if (whence == SEEK_SET) + { + assert(offset >= 0); + new_pos = offset; + } + else if (whence == SEEK_CUR) + { + new_pos = static_cast(getPosition()) + offset; + } + else + { + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Expected SEEK_SET or SEEK_CUR as whence"); + } + + /// Position is unchanged. + if (new_pos == static_cast(getPosition())) + return new_pos; + + bool read_from_prefetch = false; + while (true) + { + /// The first condition implies bytes_to_ignore = 0. + if (!working_buffer.empty() && file_offset_of_buffer_end - working_buffer.size() <= new_pos && + new_pos <= file_offset_of_buffer_end) + { + /// Position is still inside the buffer. + /// Probably it is at the end of the buffer - then we will load data on the following 'next' call. + + pos = working_buffer.end() - file_offset_of_buffer_end + new_pos; + assert(pos >= working_buffer.begin()); + assert(pos <= working_buffer.end()); + + return new_pos; + } + else if (prefetch_future.valid()) + { + read_from_prefetch = true; + + /// Read from prefetch buffer and recheck if the new position is valid inside. + if (nextImpl()) + continue; + } + + /// Prefetch is cancelled because of seek. + if (read_from_prefetch) + { + ProfileEvents::increment(ProfileEvents::RemoteFSCancelledPrefetches); + } + + break; + } + + assert(!prefetch_future.valid()); + + /// First reset the buffer so the next read will fetch new data to the buffer. + resetWorkingBuffer(); + bytes_to_ignore = 0; + + if (read_until_position && new_pos > *read_until_position) + { + ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); + file_offset_of_buffer_end = new_pos = *read_until_position; /// read_until_position is a non-included boundary. + impl->seek(file_offset_of_buffer_end, SEEK_SET); + return new_pos; + } + + /** + * Lazy ignore. Save number of bytes to ignore and ignore it either for prefetch buffer or current buffer. + * Note: we read in range [file_offset_of_buffer_end, read_until_position). + */ + if (read_until_position && new_pos < *read_until_position + && new_pos > file_offset_of_buffer_end + && new_pos < file_offset_of_buffer_end + read_settings.remote_read_min_bytes_for_seek) + { + ProfileEvents::increment(ProfileEvents::RemoteFSLazySeeks); + bytes_to_ignore = new_pos - file_offset_of_buffer_end; + } + else + { + ProfileEvents::increment(ProfileEvents::RemoteFSSeeksWithReset); + file_offset_of_buffer_end = new_pos; + impl->seek(file_offset_of_buffer_end, SEEK_SET); + } + + return new_pos; +} + + +void AsynchronousBoundedReadBuffer::finalize() +{ + resetPrefetch(FilesystemPrefetchState::UNNEEDED); +} + +AsynchronousBoundedReadBuffer::~AsynchronousBoundedReadBuffer() +{ + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void AsynchronousBoundedReadBuffer::resetPrefetch(FilesystemPrefetchState state) +{ + if (!prefetch_future.valid()) + return; + + auto [size, offset] = prefetch_future.get(); + prefetch_future = {}; + last_prefetch_info = {}; + + ProfileEvents::increment(ProfileEvents::RemoteFSPrefetchedBytes, size); + + switch (state) + { + case FilesystemPrefetchState::UNNEEDED: + ProfileEvents::increment(ProfileEvents::RemoteFSUnusedPrefetches); + break; + case FilesystemPrefetchState::CANCELLED_WITH_SEEK: + case FilesystemPrefetchState::CANCELLED_WITH_RANGE_CHANGE: + ProfileEvents::increment(ProfileEvents::RemoteFSCancelledPrefetches); + break; + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected state of prefetch: {}", magic_enum::enum_name(state)); + } +} + +} diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h new file mode 100644 index 00000000000..a70819d4efa --- /dev/null +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -0,0 +1,94 @@ +#pragma once + +#include +#include +#include +#include + +#include + +namespace DB +{ + +struct AsyncReadCounters; +using AsyncReadCountersPtr = std::shared_ptr; +class ReadBufferFromRemoteFSGather; + +class AsynchronousBoundedReadBuffer : public ReadBufferFromFileBase +{ +public: + using Impl = ReadBufferFromFileBase; + using ImplPtr = std::unique_ptr; + + explicit AsynchronousBoundedReadBuffer( + ImplPtr impl_, + IAsynchronousReader & reader_, + const ReadSettings & settings_, + AsyncReadCountersPtr async_read_counters_ = nullptr, + FilesystemReadPrefetchesLogPtr prefetches_log_ = nullptr); + + ~AsynchronousBoundedReadBuffer() override; + + String getFileName() const override { return impl->getFileName(); } + + size_t getFileSize() override { return impl->getFileSize(); } + + String getInfoForLog() override { return impl->getInfoForLog(); } + + off_t seek(off_t offset_, int whence) override; + + void prefetch() override; + + void setReadUntilPosition(size_t position) override; /// [..., position). + + void setReadUntilEnd() override { return setReadUntilPosition(getFileSize()); } + + off_t getPosition() override { return file_offset_of_buffer_end - available() + bytes_to_ignore; } + +private: + const ImplPtr impl; + const ReadSettings read_settings; + IAsynchronousReader & reader; + + size_t file_offset_of_buffer_end = 0; + std::optional read_until_position; + /// If nonzero then working_buffer is empty. + /// If a prefetch is in flight, the prefetch task has been instructed to ignore this many bytes. + size_t bytes_to_ignore = 0; + + Memory<> prefetch_buffer; + std::future prefetch_future; + + const std::string query_id; + const std::string current_reader_id; + + Poco::Logger * log; + + AsyncReadCountersPtr async_read_counters; + FilesystemReadPrefetchesLogPtr prefetches_log; + + struct LastPrefetchInfo + { + std::chrono::system_clock::time_point submit_time; + Priority priority; + }; + LastPrefetchInfo last_prefetch_info; + + bool nextImpl() override; + + void finalize(); + + bool hasPendingDataToRead(); + + void appendToPrefetchLog( + FilesystemPrefetchState state, + int64_t size, + const std::unique_ptr & execution_watch); + + std::future asyncReadInto(char * data, size_t size, Priority priority); + + void resetPrefetch(FilesystemPrefetchState state); + +}; + +} diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 6e3ebb43e50..b590ba93d84 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -491,7 +491,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( std::vector part_tags; - size_t upload_part_size = settings_ptr->request_settings.min_upload_part_size; + size_t upload_part_size = settings_ptr->request_settings.getUploadSettings().min_upload_part_size; for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size) { ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 8a3a00929db..c154e64307c 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -34,22 +34,7 @@ extern const int BAD_ARGUMENTS; std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { const Settings & settings = context->getSettingsRef(); - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries); - request_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size); - request_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor); - request_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold); - request_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size); - request_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload); - request_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries); - - // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = config.getUInt64(config_prefix + ".s3_max_get_rps", settings.s3_max_get_rps)) - request_settings.get_request_throttler = std::make_shared( - max_get_rps, config.getUInt64(config_prefix + ".s3_max_get_burst", settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); - if (UInt64 max_put_rps = config.getUInt64(config_prefix + ".s3_max_put_rps", settings.s3_max_put_rps)) - request_settings.put_request_throttler = std::make_shared( - max_put_rps, config.getUInt64(config_prefix + ".s3_max_put_burst", settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); + S3Settings::RequestSettings request_settings(config, config_prefix, settings); return std::make_unique( request_settings, @@ -122,7 +107,7 @@ std::shared_ptr getProxyConfiguration(const String & pre } -std::unique_ptr getClient( +std::unique_ptr getClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, @@ -158,6 +143,18 @@ std::unique_ptr getClient( client_configuration.retryStrategy = std::make_shared(config.getUInt(config_prefix + ".retry_attempts", 10)); + /* + * const PocoHTTPClientConfiguration & cfg_, + bool is_virtual_hosted_style, + const String & access_key_id, + const String & secret_access_key, + const String & server_side_encryption_customer_key_base64, + ServerSideEncryptionKMSConfig sse_kms_config, + HTTPHeaderEntries headers, + CredentialsConfiguration credentials_configuration, + const String & session_token) + + */ return S3::ClientFactory::instance().create( client_configuration, uri.is_virtual_hosted_style, @@ -165,8 +162,11 @@ std::unique_ptr getClient( config.getString(config_prefix + ".secret_access_key", ""), config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), {}, - config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), - config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false))); + {}, + { + .use_environment_credentials = config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)), + .use_insecure_imds_request = config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)) + }); } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 04eb7aced8e..9baa317643b 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -4,6 +4,7 @@ #if USE_AWS_S3 +#include #include #include @@ -22,7 +23,7 @@ struct S3ObjectStorageSettings; std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); +std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); } diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index e848229517a..dfbf42f7d0c 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -24,22 +24,6 @@ namespace DB::S3 namespace { -Aws::S3::Model::HeadObjectOutcome headObject( - const S3::Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) -{ - ProfileEvents::increment(ProfileEvents::S3HeadObject); - if (for_disk_s3) - ProfileEvents::increment(ProfileEvents::DiskS3HeadObject); - - S3::HeadObjectRequest req; - req.SetBucket(bucket); - req.SetKey(key); - - if (!version_id.empty()) - req.SetVersionId(version_id); - - return client.HeadObject(req); -} /// Performs a request to get the size and last modification time of an object. std::pair, Aws::S3::S3Error> tryGetObjectInfo( @@ -60,6 +44,24 @@ std::pair, Aws::S3::S3Error> tryGetObjectInfo( return {object_info, {}}; } + +} + +Aws::S3::Model::HeadObjectOutcome headObject( + const S3::Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3) +{ + ProfileEvents::increment(ProfileEvents::S3HeadObject); + if (for_disk_s3) + ProfileEvents::increment(ProfileEvents::DiskS3HeadObject); + + S3::HeadObjectRequest req; + req.SetBucket(bucket); + req.SetKey(key); + + if (!version_id.empty()) + req.SetVersionId(version_id); + + return client.HeadObject(req); } diff --git a/src/IO/S3/getObjectInfo.h b/src/IO/S3/getObjectInfo.h index a57d807644b..eae94d8e094 100644 --- a/src/IO/S3/getObjectInfo.h +++ b/src/IO/S3/getObjectInfo.h @@ -3,9 +3,9 @@ #include "config.h" #if USE_AWS_S3 -#include -#include -#include +# include +# include +# include namespace DB::S3 @@ -19,6 +19,9 @@ struct ObjectInfo std::map metadata = {}; /// Set only if getObjectInfo() is called with `with_metadata = true`. }; +Aws::S3::Model::HeadObjectOutcome +headObject(const S3::Client & client, const String & bucket, const String & key, const String & version_id, bool for_disk_s3); + ObjectInfo getObjectInfo( const S3::Client & client, const String & bucket, diff --git a/src/IO/SeekableReadBuffer.cpp b/src/IO/SeekableReadBuffer.cpp new file mode 100644 index 00000000000..ea8d4b87b0b --- /dev/null +++ b/src/IO/SeekableReadBuffer.cpp @@ -0,0 +1,63 @@ +#include + + +namespace DB +{ + +namespace +{ +template +class SeekableReadBufferWrapper : public SeekableReadBuffer +{ +public: + SeekableReadBufferWrapper(SeekableReadBuffer & in_, CustomData && custom_data_) + : SeekableReadBuffer(in_.buffer().begin(), in_.buffer().size(), in_.offset()) + , in(in_) + , custom_data(std::move(custom_data_)) + { + } + +private: + SeekableReadBuffer & in; + CustomData custom_data; + + bool nextImpl() override + { + in.position() = position(); + if (!in.next()) + { + set(in.position(), 0); + return false; + } + BufferBase::set(in.buffer().begin(), in.buffer().size(), in.offset()); + return true; + } + + off_t seek(off_t off, int whence) override + { + in.position() = position(); + off_t new_pos = in.seek(off, whence); + BufferBase::set(in.buffer().begin(), in.buffer().size(), in.offset()); + return new_pos; + } + + off_t getPosition() override + { + in.position() = position(); + return in.getPosition(); + } +}; +} + + +std::unique_ptr wrapSeekableReadBufferReference(SeekableReadBuffer & ref) +{ + return std::make_unique>(ref, nullptr); +} + +std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBufferPtr ptr) +{ + return std::make_unique>(*ptr, SeekableReadBufferPtr{ptr}); +} + +} diff --git a/src/IO/SeekableReadBuffer.h b/src/IO/SeekableReadBuffer.h index b4d5779d423..d038003f923 100644 --- a/src/IO/SeekableReadBuffer.h +++ b/src/IO/SeekableReadBuffer.h @@ -67,6 +67,37 @@ class SeekableReadBuffer : public ReadBuffer virtual bool isIntegratedWithFilesystemCache() const { return false; } }; +/// Useful for reading in parallel. +/// The created read buffers may outlive the factory. +/// +/// There are 2 ways to use this: +/// (1) Never call seek() or getFileSize(), read the file sequentially. +/// For HTTP, this usually translates to just one HTTP request. +/// (2) Call checkIfActuallySeekable(), then: +/// a. If it returned false, go to (1). seek() and getFileSize() are not available (throw if called). +/// b. If it returned true, seek() and getFileSize() are available, knock yourself out. +/// For HTTP, checkIfActuallySeekable() sends a HEAD request and returns false if the web server +/// doesn't support ranges (or doesn't support HEAD requests). +class SeekableReadBufferFactory : public WithFileSize +{ +public: + ~SeekableReadBufferFactory() override = default; + + // We usually call setReadUntilPosition() and seek() on the returned buffer before reading. + // So it's recommended that the returned implementation be lazy, i.e. don't start reading + // before the first call to nextImpl(). + virtual std::unique_ptr getReader() = 0; + + virtual bool checkIfActuallySeekable() { return true; } +}; + using SeekableReadBufferPtr = std::shared_ptr; +using SeekableReadBufferFactoryPtr = std::unique_ptr; + +/// Wraps a reference to a SeekableReadBuffer into an unique pointer to SeekableReadBuffer. +/// This function is like wrapReadBufferReference() but for SeekableReadBuffer. +std::unique_ptr wrapSeekableReadBufferReference(SeekableReadBuffer & ref); +std::unique_ptr wrapSeekableReadBufferPointer(SeekableReadBufferPtr ptr); + } diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index a0f381ed2bf..e443dfff33b 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include #include @@ -71,7 +72,7 @@ WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::RequestSettings & request_settings_, + const S3Settings::RequestSettings & request_settings, std::optional> object_metadata_, size_t buffer_size_, ThreadPoolCallbackRunner schedule_, @@ -79,10 +80,12 @@ WriteBufferFromS3::WriteBufferFromS3( : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) , key(key_) - , request_settings(request_settings_) + , settings(request_settings.getUploadSettings()) + , check_objects_after_upload(request_settings.check_objects_after_upload) + , max_unexpected_write_error_retries(request_settings.max_unexpected_write_error_retries) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , upload_part_size(request_settings_.min_upload_part_size) + , upload_part_size(settings.min_upload_part_size) , schedule(std::move(schedule_)) , write_settings(write_settings_) { @@ -91,7 +94,7 @@ WriteBufferFromS3::WriteBufferFromS3( void WriteBufferFromS3::nextImpl() { - if (!offset()) + if (offset() == 0u) return; /// Buffer in a bad state after exception @@ -111,7 +114,7 @@ void WriteBufferFromS3::nextImpl() write_settings.remote_throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. - if (multipart_upload_id.empty() && last_part_size > request_settings.max_single_part_upload_size) + if (multipart_upload_id.empty() && last_part_size > settings.max_single_part_upload_size) createMultipartUpload(); if (!multipart_upload_id.empty() && last_part_size > upload_part_size) @@ -176,7 +179,7 @@ void WriteBufferFromS3::finalizeImpl() if (!multipart_upload_id.empty()) completeMultipartUpload(); - if (request_settings.check_objects_after_upload) + if (check_objects_after_upload) { LOG_TRACE(log, "Checking object {} exists after upload", key); @@ -302,6 +305,20 @@ void WriteBufferFromS3::writePart() void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & req, int part_number) { + /// Increase part number. + ++part_number; + if (!multipart_upload_id.empty() && (part_number > settings.max_part_number)) + { + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, + "Part number exceeded {} while writing {} bytes to S3. Check min_upload_part_size = {}, max_upload_part_size = {}, " + "upload_part_size_multiply_factor = {}, upload_part_size_multiply_parts_count_threshold = {}, max_single_part_upload_size = {}", + settings.max_part_number, count(), settings.min_upload_part_size, settings.max_upload_part_size, + settings.upload_part_size_multiply_factor, settings.upload_part_size_multiply_parts_count_threshold, + settings.max_single_part_upload_size); + } + + /// Setup request. req.SetBucket(bucket); req.SetKey(key); req.SetPartNumber(part_number); @@ -311,6 +328,13 @@ void WriteBufferFromS3::fillUploadRequest(Aws::S3::Model::UploadPartRequest & re /// If we don't do it, AWS SDK can mistakenly set it to application/xml, see https://github.com/aws/aws-sdk-cpp/issues/1840 req.SetContentType("binary/octet-stream"); + + /// Maybe increase `upload_part_size` (we need to increase it sometimes to keep `part_number` less or equal than `max_part_number`). + if (!multipart_upload_id.empty() && (part_number % settings.upload_part_size_multiply_parts_count_threshold == 0)) + { + upload_part_size *= settings.upload_part_size_multiply_factor; + upload_part_size = std::min(upload_part_size, settings.max_upload_part_size); + } } void WriteBufferFromS3::processUploadRequest(UploadPartTask & task) @@ -356,7 +380,7 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); @@ -462,7 +486,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) { - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index ebc75dedd09..4b5272b3a07 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -50,7 +50,7 @@ class WriteBufferFromS3 final : public BufferWithOwnMemory std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::RequestSettings & request_settings_, + const S3Settings::RequestSettings & request_settings, std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, ThreadPoolCallbackRunner schedule_ = {}, @@ -88,7 +88,9 @@ class WriteBufferFromS3 final : public BufferWithOwnMemory const String bucket; const String key; - const S3Settings::RequestSettings request_settings; + const S3Settings::RequestSettings::PartUploadSettings settings; + const bool check_objects_after_upload = false; + const size_t max_unexpected_write_error_retries = 4; const std::shared_ptr client_ptr; const std::optional> object_metadata; diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp new file mode 100644 index 00000000000..d1cc61b94ba --- /dev/null +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -0,0 +1,61 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList FilesystemReadPrefetchesLogElement::getNamesAndTypes() +{ + return { + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"query_id", std::make_shared()}, + {"path", std::make_shared()}, + {"offset", std::make_shared()}, + {"size", std::make_shared()}, + {"prefetch_submit_time", std::make_shared(6)}, + {"priority", std::make_shared()}, + {"prefetch_execution_start_time", std::make_shared(6)}, + {"prefetch_execution_end_time", std::make_shared(6)}, + {"prefetch_execution_time_us", std::make_shared()}, + {"state", std::make_shared()}, /// Was this prefetch used or we downloaded it in vain? + {"thread_id", std::make_shared()}, + {"reader_id", std::make_shared()}, + }; +} + +void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) const +{ + size_t i = 0; + + columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); + columns[i++]->insert(event_time); + columns[i++]->insert(query_id); + columns[i++]->insert(path); + columns[i++]->insert(offset); + columns[i++]->insert(size); + columns[i++]->insert(std::chrono::duration_cast(prefetch_submit_time.time_since_epoch()).count()); + columns[i++]->insert(priority.value); + if (execution_watch) + { + columns[i++]->insert(execution_watch->getStart() / 1000); + columns[i++]->insert(execution_watch->getEnd() / 1000); + columns[i++]->insert(execution_watch->elapsedMicroseconds()); + } + else + { + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + columns[i++]->insertDefault(); + } + columns[i++]->insert(magic_enum::enum_name(state)); + columns[i++]->insert(thread_id); + columns[i++]->insert(reader_id); +} + +} diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h new file mode 100644 index 00000000000..313c8ab5872 --- /dev/null +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +enum class FilesystemPrefetchState +{ + USED, + CANCELLED_WITH_SEEK, + CANCELLED_WITH_RANGE_CHANGE, + UNNEEDED, +}; + +struct FilesystemReadPrefetchesLogElement +{ + time_t event_time{}; + String query_id; + String path; + UInt64 offset; + Int64 size; /// -1 means unknown + std::chrono::system_clock::time_point prefetch_submit_time; + std::optional execution_watch; + Priority priority; + FilesystemPrefetchState state; + UInt64 thread_id; + String reader_id; + + static std::string name() { return "FilesystemReadPrefetchesLog"; } + + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } + + void appendToBlock(MutableColumns & columns) const; + static const char * getCustomColumnList() { return nullptr; } +}; + +class FilesystemReadPrefetchesLog : public SystemLog +{ +public: + using SystemLog::SystemLog; +}; + +using FilesystemReadPrefetchesLogPtr = std::shared_ptr; + +} diff --git a/src/Parsers/ASTDataType.cpp b/src/Parsers/ASTDataType.cpp new file mode 100644 index 00000000000..a0e4cd723ac --- /dev/null +++ b/src/Parsers/ASTDataType.cpp @@ -0,0 +1,67 @@ +#include +#include +#include + + +namespace DB +{ + +String ASTDataType::getID(char delim) const +{ + return "DataType" + (delim + name); +} + +ASTPtr ASTDataType::clone() const +{ + auto res = std::make_shared(*this); + res->children.clear(); + + if (arguments) + { + res->arguments = arguments->clone(); + res->children.push_back(res->arguments); + } + + return res; +} + +void ASTDataType::updateTreeHashImpl(SipHash & hash_state) const +{ + hash_state.update(name.size()); + hash_state.update(name); + /// Children are hashed automatically. +} + +void ASTDataType::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const +{ + settings.ostr << (settings.hilite ? hilite_function : "") << name; + + if (arguments && !arguments->children.empty()) + { + settings.ostr << '(' << (settings.hilite ? hilite_none : ""); + + if (!settings.one_line && settings.print_pretty_type_names && name == "Tuple") + { + ++frame.indent; + std::string indent_str = settings.one_line ? "" : "\n" + std::string(4 * frame.indent, ' '); + for (size_t i = 0, size = arguments->children.size(); i < size; ++i) + { + if (i != 0) + settings.ostr << ','; + settings.ostr << indent_str; + arguments->children[i]->formatImpl(settings, state, frame); + } + } + else + { + frame.expression_list_prepend_whitespace = false; + arguments->formatImpl(settings, state, frame); + } + + settings.ostr << (settings.hilite ? hilite_function : "") << ')'; + } + + settings.ostr << (settings.hilite ? hilite_none : ""); +} + +} diff --git a/src/Parsers/ASTDataType.h b/src/Parsers/ASTDataType.h new file mode 100644 index 00000000000..f8fc4e16a0d --- /dev/null +++ b/src/Parsers/ASTDataType.h @@ -0,0 +1,40 @@ +#pragma once + +#include + + +namespace DB +{ + +/// AST for data types, e.g. UInt8 or Tuple(x UInt8, y Enum(a = 1)) +class ASTDataType : public IAST +{ +public: + String name; + ASTPtr arguments; + + String getID(char delim) const override; + ASTPtr clone() const override; + void updateTreeHashImpl(SipHash & hash_state) const override; + +protected: + void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; +}; + +template +std::shared_ptr makeASTDataType(const String & name, Args &&... args) +{ + auto data_type = std::make_shared(); + data_type->name = name; + + if constexpr (sizeof...(args)) + { + data_type->arguments = std::make_shared(); + data_type->children.push_back(data_type->arguments); + data_type->arguments->children = { std::forward(args)... }; + } + + return data_type; +} + +} diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 3a92cec025a..454fb0f05d0 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -170,4 +170,7 @@ class IOutputFormat : public IProcessor size_t result_rows = 0; size_t result_bytes = 0; }; + +using OutputFormatPtr = std::shared_ptr; + } diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 2af060fe4e1..b846d8329e5 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -650,7 +650,7 @@ namespace DB { arrow_fields.reserve(header.columns()); header_columns.reserve(header.columns()); - for (auto column : header.getColumnsWithTypeAndName()) + for (size_t field_id = 1; auto column : header.getColumnsWithTypeAndName()) { if (!low_cardinality_as_dictionary) { @@ -659,7 +659,12 @@ namespace DB } bool is_column_nullable = false; auto arrow_type = getArrowType(column.type, column.column, column.name, format_name, &is_column_nullable); - arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable)); + /// proton: starts + /// FIXME field_id is needed for Iceberg, we need to ensure that the field_ids match the ones defined in the iceberg table schema. + auto kv_metadata = std::make_shared(); + kv_metadata->Append("PARQUET:field_id", fmt::format("{}", field_id)); /// field_id starts from 1 + arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable, std::move(kv_metadata))); + /// proton: ends header_columns.emplace_back(std::move(column)); } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 57819eb7fd0..fa39144d919 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -39,9 +39,12 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) auto sink = std::make_shared(out); parquet::WriterProperties::Builder builder; -#if USE_SNAPPY + /// proton: FIXME + /// Make compression configurable. +#if 0 /// USE_SNAPPY builder.compression(parquet::Compression::SNAPPY); #endif + builder.compression(parquet::Compression::ZSTD); auto props = builder.build(); auto result = parquet::arrow::FileWriter::Open( *arrow_table->schema(), diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index c0421a4d99f..c8017de49e6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -33,6 +33,8 @@ class ParquetBlockOutputFormat : public IOutputFormat String getContentType() const override { return "application/octet-stream"; } + const parquet::arrow::FileWriter & fileWriter() const { return *file_writer; } /// proton: added + private: void consume(Chunk) override; void finalizeImpl() override; diff --git a/src/Processors/ProcessorID.h b/src/Processors/ProcessorID.h index f1510305c70..77480324bf3 100644 --- a/src/Processors/ProcessorID.h +++ b/src/Processors/ProcessorID.h @@ -249,15 +249,16 @@ enum class ProcessorID : UInt32 GenerateRandomSourceID = 10'045, SourceFromQueryPipelineID = 10'046, ConvertingAggregatedToChunksSourceShuffledID = 10'047, - /// proton: starts + ClickHouseSourceID = 11'000, PulsarSourceID = 11'054, - /// proton: ends + IcebergSourceID = 11'100, /// Sink Processors EmptySinkID = 20'000, NullSinkID = 20'001, ExternalTableDataSinkID = 20'002, + IcebergSinkID = 20'100, }; inline ProcessorID toProcessID(UInt32 v) diff --git a/src/Storages/ExternalStream/CMakeLists.txt b/src/Storages/ExternalStream/CMakeLists.txt index a74389d9973..11b8db28888 100644 --- a/src/Storages/ExternalStream/CMakeLists.txt +++ b/src/Storages/ExternalStream/CMakeLists.txt @@ -6,6 +6,7 @@ add_headers_and_sources(external_stream Log) add_headers_and_sources(external_stream Kafka) add_headers_and_sources(external_stream Pulsar) add_headers_and_sources(external_stream Timeplus) +add_headers_and_sources(external_stream Iceberg) add_library(external_stream ${external_stream_headers} ${external_stream_sources}) @@ -19,6 +20,14 @@ if (TARGET ch_contrib::pulsar) target_link_libraries(external_stream PRIVATE ch_contrib::pulsar) endif() +if (TARGET ch_contrib::avrocpp) + target_link_libraries(external_stream PRIVATE ch_contrib::avrocpp) +endif() + +if (TARGET ch_contrib::parquet) + target_link_libraries(external_stream PRIVATE ch_contrib::parquet) +endif() + if (ENABLE_TESTS) add_subdirectory(tests) endif () diff --git a/src/Storages/ExternalStream/ExternalStreamSettings.h b/src/Storages/ExternalStream/ExternalStreamSettings.h index 135cac4ce94..a709c995863 100644 --- a/src/Storages/ExternalStream/ExternalStreamSettings.h +++ b/src/Storages/ExternalStream/ExternalStreamSettings.h @@ -27,6 +27,8 @@ class ASTStorage; M(String, message_key, "", "(Deprecated) An expression which will be evaluated on each row of data returned by the query to compute a string which will be used as the message key. This setting is deprecated, please define a `_tp_message_key` column in the external stream instead.", 0) \ M(Bool, one_message_per_row, false, "If set to true, when send data to the Kafka external stream with row-based data format like `JSONEachRow`, it will produce one message per row.", 0) \ M(String, region, "", "The AWS region to target.", 0) \ + M(String, access_key_id, "", "The access key ID.", 0) \ + M(String, secret_access_key, "", "The secret access key.", 0) \ M(Bool, log_stats, false, "If set to true, print statistics to the logs. Note that, the statistics could contain quite a lot of data. The frequency of the statistics logs is control by the statistics.interval.ms property.", 0) \ M(Milliseconds, consumer_stall_timeout_ms, 60 * 1000, "Define the amount of time when a consumer is not making any progress, then consider the consumer stalled, and then a new consumer will be created. Adjust the value based on how busy a topic is. Use small values for a busy topic to avoid big latency. Use big values for less busy topics to avoid disruption. Set to 0 to disable the behavior.", 0) @@ -59,13 +61,17 @@ class ASTStorage; M(UInt64, memory_limit, 0, "Configure a limit on the amount of memory that will be allocated by this external stream. Setting this to 0 will disable the limit. By default this is disabled.", 0) \ M(UInt64, io_threads, 1, "Set the number of IO threads to be used by the Pulsar client. Default is 1 thread.", 0) +#define ICEBERG_EXTERNAL_STREAM_SETTINGS(M) \ + M(String, iceberg_storage_endpoint, "", "Endpoint for data storage.", 0) + #define ALL_EXTERNAL_STREAM_SETTINGS(M) \ M(String, type, "", "External stream type", 0) \ M(String, config_file, "", "External stream configuration file path", 0) \ KAFKA_EXTERNAL_STREAM_SETTINGS(M) \ LOG_FILE_EXTERNAL_STREAM_SETTINGS(M) \ TIMEPLUS_EXTERNAL_STREAM_SETTINGS(M) \ - PULSAR_EXTERNAL_STREAM_SETTINGS(M) + PULSAR_EXTERNAL_STREAM_SETTINGS(M) \ + ICEBERG_EXTERNAL_STREAM_SETTINGS(M) #define LIST_OF_EXTERNAL_STREAM_SETTINGS(M) \ ALL_EXTERNAL_STREAM_SETTINGS(M) \ diff --git a/src/Storages/ExternalStream/ExternalStreamTypes.h b/src/Storages/ExternalStream/ExternalStreamTypes.h index aec5f56fd48..0bda1a3472f 100644 --- a/src/Storages/ExternalStream/ExternalStreamTypes.h +++ b/src/Storages/ExternalStream/ExternalStreamTypes.h @@ -9,5 +9,6 @@ namespace StreamTypes const String PULSAR = "pulsar"; const String TIMEPLUS = "timeplus"; const String LOG = "log"; + const String ICEBERG = "iceberg"; } } diff --git a/src/Storages/ExternalStream/Iceberg/Iceberg.cpp b/src/Storages/ExternalStream/Iceberg/Iceberg.cpp new file mode 100644 index 00000000000..b5553cc6223 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/Iceberg.cpp @@ -0,0 +1,352 @@ +#include + +#if USE_AVRO + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int INVALID_SETTING_VALUE; +} + +namespace +{ + +void updateFormatFactorySettings(FormatFactorySettings & settings, const ContextPtr & context) +{ + const auto & changes = context->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (settings.has(change.name)) + settings.set(change.name, change.value); + } +} + +} + +namespace ExternalStream +{ + +Iceberg::Iceberg(IStorage * storage, ExternalStreamSettingsPtr settings_, ExternalStreamCounterPtr, ContextPtr context) + : StorageExternalStreamImpl(storage, std::move(settings_), context) +{ + auto default_virtuals = NamesAndTypesList{ + {"_path", std::make_shared(std::make_shared())}, + {"_file", std::make_shared(std::make_shared())}}; + + auto columns = storage->getInMemoryMetadata().getSampleBlock().getNamesAndTypesList(); + virtual_columns = getVirtualsForStorage(columns, default_virtuals); + for (const auto & column : virtual_columns) + virtual_block.insert({column.type->createColumn(), column.type, column.name}); + + IcebergS3Configuration configuration; + + auto storage_endpoint = settings->iceberg_storage_endpoint.value; + if (storage_endpoint.empty()) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "iceberg_storage_endpoint cannot be empty"); + + configuration.url = S3::URI(storage_endpoint); + configuration.format = "Parquet"; + + configuration.min_upload_file_size = context->getSettingsRef().s3_min_upload_file_size; + configuration.max_upload_idle_seconds = context->getSettingsRef().s3_max_upload_idle_seconds; + + configuration.auth_settings.access_key_id = settings->access_key_id; + configuration.auth_settings.secret_access_key = settings->secret_access_key; + configuration.auth_settings.region = configuration.url.region; + configuration.auth_settings.use_environment_credentials = true; ///settings->use_environment_credentials; + + configuration.request_settings.updateFromSettings(context->getSettingsRef()); + + s3_configuration = std::move(configuration); + s3_configuration.createClient(context); +} + +Apache::Iceberg::CatalogPtr Iceberg::getCatalog() const +{ + auto database = DatabaseCatalog::instance().getDatabase(getStorageID().database_name); + auto iceberg_db = std::dynamic_pointer_cast(database); + assert(iceberg_db); + + return iceberg_db->getCatalog(); +} + +Apache::Iceberg::TableMetadata Iceberg::getTableMetadata() const +{ + auto storage_id = getStorageID(); + /// TODO withCredentials() + Apache::Iceberg::TableMetadata metadata; + metadata.withSchema().withLocation(); + getCatalog()->getTableMetadata(storage_id.database_name, storage_id.table_name, metadata); + + return metadata; +} + +std::list Iceberg::fetchManifestList(const Apache::Iceberg::TableMetadata & table_metadata) const +{ + std::list manifest_lists; + + const auto & manifest_list_uri = table_metadata.getManifestList(); + if (manifest_list_uri.empty()) + { + LOG_INFO(logger, "Table metadata does not have a manifest list."); + return manifest_lists; + } + + ReadBufferFromS3 buf{ + s3_configuration.client, + s3_configuration.url.bucket, + Poco::URI(manifest_list_uri).getPath(), + /*version_id_=*/"", + s3_configuration.request_settings, + /*settings_=*/{}}; + + auto is = std::make_unique(buf); + + auto schema = avro::compileJsonSchemaFromString(Apache::Iceberg::AvroSchemas::MANIFEST_LIST); + avro::DataFileReader reader{std::move(is), schema}; + + Apache::Iceberg::ManifestList manifest_list; + auto enc = avro::jsonEncoder(schema); + while (reader.read(manifest_list)) + { + LOG_INFO(logger, "Got manifest_list sn = {} sid = {}", manifest_list.sequence_number, manifest_list.added_snapshot_id); + manifest_lists.push_back(std::move(manifest_list)); + } + + return manifest_lists; +} + +void Iceberg::fetchDataFiles(const Apache::Iceberg::ManifestList & manifest_list, std::list & data_files) const +{ + LOG_INFO(logger, "Fetching manifest from {}", manifest_list.manifest_path); + ReadBufferFromS3 buf{ + s3_configuration.client, + s3_configuration.url.bucket, + Poco::URI(manifest_list.manifest_path).getPath(), + /*version_id_=*/"", + s3_configuration.request_settings, + /*settings_=*/{}}; + + auto is = std::make_unique(buf); + + auto schema = avro::compileJsonSchemaFromString(Apache::Iceberg::AvroSchemas::MANIFEST); + avro::DataFileReader reader{std::move(is), schema}; + + Apache::Iceberg::Manifest manifest; + auto enc = avro::jsonEncoder(schema); + while (reader.read(manifest)) + { + LOG_INFO(logger, "Got manifest data_file = {}", manifest.data_file.file_path); + if (!boost::iequals(manifest.data_file.file_format, "parquet")) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Data file format {} is not supported", manifest.data_file.file_format); + if (!manifest.data_file.file_path.empty()) + data_files.push_back(std::move(manifest.data_file.file_path)); + } +} + +namespace +{ +std::shared_ptr createFileIterator( + Strings keys, + IcebergS3Configuration s3_configuration, + bool distributed_processing, + ContextPtr local_context, + ASTPtr query, + const Block & virtual_block, + IcebergSource::ObjectInfos * object_infos) +{ + if (distributed_processing) + { + return std::make_shared(local_context->getReadTaskCallback()); + } + else + { + return std::make_shared( + s3_configuration.client, + s3_configuration.url.version_id, + keys, + s3_configuration.url.bucket, + s3_configuration.request_settings, + query, + virtual_block, + local_context, + object_infos); + } +} +} + +Pipe Iceberg::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + auto table_metadata = getTableMetadata(); + auto manifest_lists = fetchManifestList(table_metadata); + + auto header = storage_snapshot->getSampleBlockForColumns(column_names); + + Pipes pipes; + + if (manifest_lists.empty()) + { /// No data in the table + LOG_INFO(logger, "No manifest list, no data to read"); + pipes.reserve(1); + pipes.emplace_back(std::make_shared(header)); + } + else + { + std::unordered_set column_names_set(column_names.begin(), column_names.end()); + std::vector requested_virtual_columns; + + for (const auto & virtual_column : getVirtuals()) + { + if (column_names_set.contains(virtual_column.name)) + requested_virtual_columns.push_back(virtual_column); + } + + std::list data_files; + for (const auto & manifest_list : manifest_lists) + fetchDataFiles(manifest_list, data_files); + + LOG_INFO(logger, "Reading {} data files", data_files.size()); + + Strings keys; + keys.reserve(data_files.size()); + for (const auto & data_file : data_files) + keys.push_back(Poco::URI(data_file).getPath()); + + std::shared_ptr iterator_wrapper = createFileIterator( + keys, + s3_configuration, + /*distributed_processing=*/false, + local_context, + query_info.query, + virtual_block, + &object_infos); + + ColumnsDescription columns_description; + Block block_for_format; + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if(fetch_columns, [&](const String & col) { + return std::any_of( + virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col) { return col == virtual_col.name; }); + }); + + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } + + const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + requested_virtual_columns, + "Parquet", /// for now, only parquet data files are supported + getName(), + block_for_format, + local_context, + getFormatSettings(local_context), + columns_description, + max_block_size, + s3_configuration.request_settings, + /*compression_method=*/"none", + s3_configuration.client, + s3_configuration.url.bucket, + s3_configuration.url.version_id, + iterator_wrapper, + max_download_threads)); + } + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + + return pipe; +} + +SinkToStoragePtr Iceberg::write(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) +{ + auto table_metadata = getTableMetadata(); + auto manifest_lists = fetchManifestList(table_metadata); + + auto min_upload_file_size_ = local_context->getSettingsRef().s3_min_upload_file_size.changed + ? local_context->getSettingsRef().s3_min_upload_file_size.value + : s3_configuration.min_upload_file_size; + + auto max_upload_idle_seconds_ = local_context->getSettingsRef().s3_max_upload_idle_seconds.changed + ? local_context->getSettingsRef().s3_max_upload_idle_seconds.value + : s3_configuration.max_upload_idle_seconds; + + auto sample_block = metadata_snapshot->getSampleBlock(); + + auto format_settings = getFormatSettings(local_context); + + return std::make_shared( + getStorageID(), + "Parquet", + sample_block, + format_settings, + s3_configuration, + s3_configuration.url.bucket, + getStorageID().getTableName(), + min_upload_file_size_, + max_upload_idle_seconds_, + std::move(table_metadata), + std::move(manifest_lists), + getCatalog(), + local_context); +} + +FormatSettings Iceberg::getFormatSettings(const ContextPtr & local_context) const +{ + FormatFactorySettings settings = format_factory_settings.has_value() ? *format_factory_settings : FormatFactorySettings(); + updateFormatFactorySettings(settings, local_context); + auto format_settings = DB::getFormatSettings(local_context, settings); + + /// This is needed otherwise using an external stream with ProtobufSingle format as the target stream + /// of a MV (or in `INSERT ... SELECT ...`), i.e. more than one rows sent to the stream, exception will be thrown. + format_settings.protobuf.allow_multiple_rows_without_delimiter = true; + + /// This is required otherwise tools like Spark can't read data of string columns. + format_settings.parquet.output_string_as_string = true; + return format_settings; +} + +} + +} +#endif diff --git a/src/Storages/ExternalStream/Iceberg/Iceberg.h b/src/Storages/ExternalStream/Iceberg/Iceberg.h new file mode 100644 index 00000000000..84db14b47a7 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/Iceberg.h @@ -0,0 +1,71 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ExternalStream +{ + +class Iceberg final : public StorageExternalStreamImpl +{ + using ObjectInfos = IcebergSource::ObjectInfos; + +public: + Iceberg(IStorage *, ExternalStreamSettingsPtr, ExternalStreamCounterPtr, ContextPtr); + ~Iceberg() override = default; + + String getName() const override { return "IcebergExternalStream"; } + + /// For now, we only support Parquet format and it supports subset of columns. + bool supportsSubsetOfColumns() const override { return true; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; + + /// FIXME this is a temporary trick. + void setIcebergSchema(const std::string & schema) { iceberg_schema_json = schema; } + +private: + Apache::Iceberg::CatalogPtr getCatalog() const; + Apache::Iceberg::TableMetadata getTableMetadata() const; + std::list fetchManifestList(const Apache::Iceberg::TableMetadata &) const; + void fetchDataFiles(const Apache::Iceberg::ManifestList &, std::list & data_files) const; + + FormatSettings getFormatSettings(const ContextPtr & local_context) const; + + NamesAndTypesList virtual_columns; + Block virtual_block; + /// Temporary. FIXME + ObjectInfos object_infos; + + IcebergS3Configuration s3_configuration; + std::optional format_factory_settings; + + std::string iceberg_schema_json; +}; + +} + +} + +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.cpp b/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.cpp new file mode 100644 index 00000000000..d50041ef20c --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.cpp @@ -0,0 +1,58 @@ +#include + +#if USE_AWS_S3 && USE_AVRO + +#include + +namespace DB +{ + +namespace ExternalStream +{ + +void IcebergS3Configuration::createClient(const ContextPtr & ctx) +{ + if (client) + return; + + request_settings.updateFromSettings(ctx->getSettings()); + + DB::S3::PocoHTTPClientConfiguration client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + auth_settings.region, + ctx->getRemoteHostFilter(), + static_cast(ctx->getGlobalContext()->getSettingsRef().s3_max_redirects), + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false, + request_settings.get_request_throttler, + request_settings.put_request_throttler); + + client_configuration.endpointOverride = url.endpoint; + client_configuration.maxConnections = static_cast(request_settings.max_connections); + + auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key); + auto headers = auth_settings.headers; + if (!headers_from_ast.empty()) + headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); + + client = DB::S3::ClientFactory::instance().create( + client_configuration, + url.is_virtual_hosted_style, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + auth_settings.server_side_encryption_customer_key_base64, + auth_settings.server_side_encryption_kms_config, + std::move(headers), + S3::CredentialsConfiguration{ + /*use_environment_credentials=*/true, + auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + auth_settings.expiration_window_seconds.value_or( + ctx->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + auth_settings.no_sign_request.value_or(ctx->getConfigRef().getBool("s3.no_sign_request", false)), + }); +} + +} + +} + +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.h b/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.h new file mode 100644 index 00000000000..901cd274153 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergS3Configuration.h @@ -0,0 +1,44 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ExternalStream +{ + +struct IcebergS3Configuration : public StatelessTableEngineConfiguration +{ + DB::S3::URI url; + std::shared_ptr client; + DB::S3::AuthSettings auth_settings; + S3Settings::RequestSettings request_settings; + /// If s3 configuration was passed from ast, then it is static. + /// If from config - it can be changed with config reload. + bool static_configuration = true; + /// Headers from ast is a part of static configuration. + HTTPHeaderEntries headers_from_ast; + + /// proton: starts + UInt64 min_upload_file_size = 0; + UInt64 max_upload_idle_seconds = 0; + /// proton: ends + /// +public: + void createClient(const ContextPtr & ctx); +}; + +} + +} + +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergSink.cpp b/src/Storages/ExternalStream/Iceberg/IcebergSink.cpp new file mode 100644 index 00000000000..7b4ec0d8761 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergSink.cpp @@ -0,0 +1,419 @@ +#include + +#if USE_AVRO && USE_PARQUET + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ +/// Adds current timestamp to the base_key (before the file extension if it has one). +String genKey(const String & prefix) +{ + static size_t count = 0; + + /// 00000-0-ad7b1fd7-6834-4ef0-956e-b8a6dc7f07e1-0-00001.parquet + auto result = std::filesystem::path(prefix) / "data" + / fmt::format("{:05}-{}-{}-0-00001.parquet", /*partition_id=*/0, count, toString(UUIDHelpers::generateV4())); + + ++count; + + return result; +} + +} + +namespace ExternalStream +{ + +IcebergSink::IcebergSink( + const StorageID & storage_id_, + const String & format_, + const Block & sample_block_, + std::optional format_settings_, + const IcebergS3Configuration & s3_configuration_, + const String & bucket_, + const String & key_, + UInt64 min_upload_file_size_, + UInt64 max_upload_idle_seconds_, + Apache::Iceberg::TableMetadata metadata_, + std::list current_manifest_lists, + const Apache::Iceberg::CatalogPtr & catalog_, + ContextPtr context_) + : SinkToStorage(sample_block_, ProcessorID::IcebergSinkID) + , storage_id(storage_id_) + , format(format_) + , sample_block(sample_block_) + , catalog(catalog_) + , metadata(std::move(metadata_)) + , manifest_lists(std::move(current_manifest_lists)) + , s3_configuration(s3_configuration_) + , bucket(bucket_) + , key(key_) + , format_settings(format_settings_) + , min_upload_file_size(min_upload_file_size_) + , max_upload_idle_seconds(max_upload_idle_seconds_) + , context(context_) + , logger(&Poco::Logger::get("IcebergSink")) +{ + /// The timer should only start when data start flowing. + upload_idle_timer.stop(); + next_callback = [this](size_t new_data_size, size_t /*parts_count*/) { current_total_size += new_data_size; }; +} + +IcebergSink::~IcebergSink() +{ + onFinish(); +} + +void IcebergSink::consume(Chunk chunk) +{ + if (!chunk.hasRows()) + { + if (max_upload_idle_seconds > 0 && upload_idle_timer.elapsedSeconds() >= max_upload_idle_seconds) + { + std::lock_guard lock(cancel_mutex); + finalize(); + } + return; + } + + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + + if (current_total_size >= min_upload_file_size) + /// Properly finalize the format writer and complete the current upload. + finalize(); + + if (!writer) + { + std::string object_key; + do + { + current_data_file_uri = genKey(metadata.getLocation(false)); + object_key = Poco::URI(current_data_file_uri).getPath(); + } while (DB::S3::objectExists(*s3_configuration.client, s3_configuration.url.bucket, object_key, s3_configuration.url.version_id)); + + write_buf = std::make_unique( + s3_configuration.client, + bucket, + object_key, + s3_configuration.request_settings, + std::nullopt, + DBMS_DEFAULT_BUFFER_SIZE, + threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), + context->getWriteSettings(), + next_callback); + writer + = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); + writer->setAutoFlush(); + } + + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + /// Restart the timer when there are new data, because we are calculating the idle time. + upload_idle_timer.start(); +} + +void IcebergSink::onCancel() +{ + std::lock_guard lock(cancel_mutex); + finalize(); + cancelled = true; + stopped = true; +} + +void IcebergSink::onException() +{ + std::lock_guard lock(cancel_mutex); + finalize(); + stopped = true; +} + +void IcebergSink::onFinish() +{ + std::lock_guard lock(cancel_mutex); + finalize(); + stopped = true; +} + +void IcebergSink::finalize() +{ + if (!writer) + return; + + /// Stop ParallelFormattingOutputFormat correctly. + SCOPE_EXIT(writer.reset()); + + + writer->finalize(); + writer->flush(); + write_buf->finalize(); + + commit(); + + current_total_size = 0; + upload_idle_timer.reset(); +} + +namespace +{ +int64_t genSnapshotID() +{ + auto uuid = UUIDHelpers::generateV4(); + const auto & high_bits = UUIDHelpers::getHighBytes(uuid); + const auto & low_bits = UUIDHelpers::getLowBytes(uuid); + return (high_bits ^ low_bits) & LLONG_MAX; +} + +auto genManifestListPath(const Apache::Iceberg::TableMetadata & metadata, int64_t snapshot_id, size_t attempts, const UUID & commit_uuid) +{ + /// Mimics the behavior in Java: + /// https://github.com/apache/iceberg/blob/c862b9177af8e2d83122220764a056f3b96fd00c/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L491 + return fmt::format("{}/metadata/snap-{}-{}-{}.avro", metadata.getLocation(false), snapshot_id, attempts, toString(commit_uuid)); +} + +auto genManifestPath(const Apache::Iceberg::TableMetadata & metadata, const UUID & commit_uuid, size_t num) +{ + return fmt::format("{}/metadata/{}-m{}.avro", metadata.getLocation(false), toString(commit_uuid), num); +} +} + +void IcebergSink::commit() +{ + /// TODO make sure that snapshot_id is not being used. + auto snapshot_id = genSnapshotID(); + auto commit_uuid = UUIDHelpers::generateV4(); + auto sequence_number = metadata.getSequenceNumber() + 1; + size_t attempts = 0; + + /// The manifest files and manifest list must be generated before calling commitTable on the catalog. + /// Otherwise, if it fails to write those files after it calls commitTable, the table will be in a bad state. + generateManifest(commit_uuid); + writeManifest(); + + /// TODO if commitTable fails later due to conflict (409), we should re-generate the manifest list (increase attempts). + generateManifestList(commit_uuid, snapshot_id, sequence_number, attempts); + writeManifestList(); + + const auto & current_manifest_list = manifest_lists.back(); + + auto total_data_files = current_manifest_list.added_files_count; + auto total_delete_files = current_manifest_list.deleted_files_count; + auto total_records = current_manifest_list.added_rows_count; + auto total_files_size = current_manifest.data_file.file_size_in_bytes; + auto total_position_deletes = 0; + auto total_equality_deletes = 0; + + const auto & current_snapshot = metadata.getCurrentSnapshot(); + + if (current_snapshot.isValid()) + { + total_data_files += std::stol(current_snapshot.summary.total_data_files); + total_delete_files += std::stol(current_snapshot.summary.total_delete_files); + total_records += std::stoll(current_snapshot.summary.total_records); + total_files_size += std::stoll(current_snapshot.summary.total_files_size); + total_position_deletes += std::stoll(current_snapshot.summary.total_position_deletes); + total_equality_deletes += std::stoll(current_snapshot.summary.total_equality_deletes); + } + + Apache::Iceberg::Snapshot snapshot{ + .snapshot_id = snapshot_id, + .parent_snapshot_id = metadata.getSnapshotID() < 0 ? std::nullopt : std::optional(metadata.getSnapshotID()), + .sequence_number = sequence_number, + .timestamp_ms = UTCMilliseconds::now(), + .manifest_list = current_manifest_list_uri, + .summary = { + /// TODO make this a enum + .operation = "append", + .added_files_size = fmt::format("{}", current_manifest.data_file.file_size_in_bytes), + .added_data_files = "1", + .added_records = fmt::format("{}", current_manifest_list.added_rows_count), + /// FIXME all total_ fields + .total_data_files = std::to_string(total_data_files), + .total_delete_files = std::to_string(total_delete_files), + .total_records = std::to_string(total_records), + .total_files_size = std::to_string(total_files_size), + .total_position_deletes = std::to_string(total_position_deletes), + .total_equality_deletes = std::to_string(total_equality_deletes), + }, + }; + + Apache::Iceberg::Updates updates; + updates.reserve(2); + updates.push_back(std::make_unique(snapshot)); + updates.push_back(std::make_unique(Apache::Iceberg::SnapshotReference{ + .name = "main", + .type = "branch", + .snapshot_id = snapshot_id, + })); + + Apache::Iceberg::Requirements requirements; + requirements.reserve(2); + requirements.push_back(std::make_unique("main", metadata.getSnapshotID())); + requirements.push_back(std::make_unique(metadata.getTableUUID())); + + /// TODO + /// Retry on failure. When retry, should re-generate the manifest list path (increase attempts). + catalog->commitTable(storage_id.getDatabaseName(), storage_id.getTableName(), requirements, updates, metadata); +} + +void IcebergSink::writeManifestList() const +{ + const auto & current_manifest_list = manifest_lists.back(); + + WriteBufferFromS3 buf{ + s3_configuration.client, + bucket, + Poco::URI(current_manifest_list_uri).getPath(), + s3_configuration.request_settings, + std::nullopt, + DBMS_DEFAULT_BUFFER_SIZE, + threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), + context->getWriteSettings()}; + + std::unordered_map file_metadata; + file_metadata.reserve(5); + file_metadata["format-version"] = fmt::format("{}", metadata.getFormatVersion()); + file_metadata["snapshot-id"] = fmt::format("{}", current_manifest_list.added_snapshot_id); + file_metadata["parent-snapshot-id"] = "null"; + file_metadata["sequence-number"] = fmt::format("{}", current_manifest_list.sequence_number); + file_metadata["iceberg.schema"] = Apache::Iceberg::AvroSchemas::MANIFEST_LIST_ICEBERG; + + Avro::DataFileWriterBase file_writer{ + std::make_unique(buf), + Apache::Iceberg::AvroSchemas::MANIFEST_LIST, + file_metadata, + /*sync_interval=*/16 * 1024, /// FIXME + avro::Codec::DEFLATE_CODEC}; + + file_writer.syncIfNeeded(); + for (const auto & manifest_list : manifest_lists) + { + avro::encode(file_writer.encoder(), manifest_list); + file_writer.incr(); + } + file_writer.flush(); + file_writer.close(); +} + +void IcebergSink::generateManifestList(const UUID & commit_uuid, int64_t snapshot_id, uint64_t sequence_number, size_t attempts) +{ + current_manifest_list_uri = genManifestListPath(metadata, snapshot_id, attempts, commit_uuid); + LOG_INFO(logger, "Generated manifest list path: {}", current_manifest_list_uri); + + Apache::Iceberg::ManifestList manifest_list; + manifest_list.manifest_path = current_manifest_uri; + manifest_list.manifest_length = current_total_size; /// FIXME + manifest_list.sequence_number = sequence_number; + manifest_list.min_sequence_number = sequence_number; + manifest_list.added_snapshot_id = snapshot_id; + manifest_list.added_files_count = 1; + manifest_list.added_rows_count = current_manifest.data_file.record_count; + manifest_list.existing_files_count = 0; + manifest_list.deleted_files_count = 0; + manifest_list.existing_rows_count = 0; + manifest_list.deleted_rows_count = 0; + + Apache::Iceberg::partitions partitions; + partitions.set_array({}); + manifest_list.partitions = std::move(partitions); + // manifest.key_metadata = nullptr; + + manifest_lists.push_front(std::move(manifest_list)); +} + +void IcebergSink::writeManifest() const +{ + WriteBufferFromS3 buf{ + s3_configuration.client, + bucket, + Poco::URI(current_manifest_uri).getPath(), + s3_configuration.request_settings, + std::nullopt, + DBMS_DEFAULT_BUFFER_SIZE, + threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), + context->getWriteSettings()}; + + std::unordered_map file_metadata; + assert(!metadata.getSchemaJSON().empty()); + file_metadata["schema"] = metadata.getSchemaJSON(); + file_metadata["format-version"] = fmt::format("{}", metadata.getFormatVersion()); + file_metadata["partition-spec-id"] = "0"; + file_metadata["partition-spec"] = "[]"; + file_metadata["content"] = "data"; + file_metadata["iceberg.schema"] = Apache::Iceberg::AvroSchemas::MANIFEST_ICEBERG; + + Avro::DataFileWriterBase file_writer{ + std::make_unique(buf), + Apache::Iceberg::AvroSchemas::MANIFEST, + file_metadata, + /*sync_interval=*/16 * 1024, /// FIXME? + avro::Codec::DEFLATE_CODEC}; + + file_writer.syncIfNeeded(); + avro::encode(file_writer.encoder(), current_manifest); + file_writer.incr(); + file_writer.flush(); + file_writer.flush(); +} + +void IcebergSink::generateManifest(const UUID & commit_uuid) +{ + current_manifest_uri = genManifestPath(metadata, commit_uuid, 0); + LOG_INFO(logger, "Generated manifest key: {}", current_manifest_uri); + + Apache::Iceberg::Manifest manifest; + /// 0: EXISTING 1: ADDED 2: DELETED + manifest.status = 1; + /// manifest.snapshot_id = {}; /// optional, inherited from manifest list + /// manifest.sequence_number = {}; /// optional, should set to null for add new files + /// manifest.file_sequence_number = {}; /// optional, should set to null for add new files + + auto parquet_output_format = std::dynamic_pointer_cast(writer); + if (!parquet_output_format) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected writer should be a ParquetBlockOutputFormat"); + + /// All other fields not assigned below in data_file are optional. + /// TODO set all the optional fields for better query planning. + manifest.data_file.file_path = current_data_file_uri; + /// avro, orc, parquet, or puffin + manifest.data_file.file_format = "PARQUET"; + manifest.data_file.partition = {}; + manifest.data_file.record_count = parquet_output_format->fileWriter().metadata()->num_rows(); + manifest.data_file.file_size_in_bytes = current_total_size; + /// 0: DATA, 1: POSITION DELETES, 2: EQUALITY DELETES + manifest.data_file.content = 0; + + current_manifest = std::move(manifest); +} + + +} +} +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergSink.h b/src/Storages/ExternalStream/Iceberg/IcebergSink.h new file mode 100644 index 00000000000..9a6ce1c7ee4 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergSink.h @@ -0,0 +1,105 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 && USE_AVRO + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ExternalStream +{ + +class IcebergSink final : public SinkToStorage +{ +public: + IcebergSink( + const StorageID & storage_id_, + const String & format_, + const Block & sample_block_, + std::optional format_settings_, + const IcebergS3Configuration & s3_configuration_, + const String & bucket_, + const String & key_, + UInt64 min_upload_file_size_, + UInt64 max_upload_idle_seconds_, + Apache::Iceberg::TableMetadata metadata_, + std::list current_manifest_lists, + const Apache::Iceberg::CatalogPtr & catalog_, + ContextPtr context_); + + ~IcebergSink() override; + + String getName() const override { return "IcebergSink"; } + + void consume(Chunk chunk) override; + void onCancel() override; + void onException() override; + void onFinish() override; + +private: + void generateManifestList(const UUID & commit_uuid, int64_t snapshot_id, uint64_t sequence_number, size_t attempts); + void writeManifestList() const; + + void generateManifest(const UUID & commit_uuid); + void writeManifest() const; + + void commit(); + + void finalize(); + + const StorageID storage_id; + const String format; + const Block sample_block; + + Apache::Iceberg::CatalogPtr catalog; + Apache::Iceberg::TableMetadata metadata; + std::list manifest_lists; + + const IcebergS3Configuration s3_configuration; + String bucket; + [[maybe_unused]] const String key; + std::optional format_settings; + + std::unique_ptr write_buf; + OutputFormatPtr writer; + + bool cancelled = false; + std::mutex cancel_mutex; + + size_t current_total_size{0}; + UInt64 min_upload_file_size{0}; + UInt64 max_upload_idle_seconds{0}; + + ASTPtr file_exprssion_ast; + + bool stopped = false; + Stopwatch upload_idle_timer; + + std::function next_callback; + + String current_data_file_uri; + String current_manifest_list_uri; + String current_manifest_uri; + + Apache::Iceberg::Manifest current_manifest; + + ContextPtr context; + Poco::Logger * logger; +}; + +} + +} + +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergSource.cpp b/src/Storages/ExternalStream/Iceberg/IcebergSource.cpp new file mode 100644 index 00000000000..a8c7c781399 --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergSource.cpp @@ -0,0 +1,425 @@ +#include + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace CurrentMetrics +{ +extern const Metric StorageS3Threads; +extern const Metric StorageS3ThreadsActive; +} + +namespace ProfileEvents +{ +extern const Event S3DeleteObjects; +extern const Event S3ListObjects; +} + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int UNEXPECTED_EXPRESSION; +} + +namespace ExternalStream +{ + +namespace +{ + +void addPathToVirtualColumns(Block & block, const String & path, size_t idx) +{ + if (block.has("_path")) + block.getByName("_path").column->assumeMutableRef().insert(path); + + if (block.has("_file")) + { + auto pos = path.find_last_of('/'); + assert(pos != std::string::npos); + + auto file = path.substr(pos + 1); + block.getByName("_file").column->assumeMutableRef().insert(file); + } + + block.getByName("_idx").column->assumeMutableRef().insert(idx); +} + +} + +class IcebergSource::KeysIterator::Impl : WithContext +{ +public: + explicit Impl( + const std::shared_ptr & client_, /// proton: updated + const std::string & version_id_, + const std::vector & keys_, + const String & bucket_, + const S3Settings::RequestSettings & request_settings_, + ASTPtr query_, + const Block & virtual_header_, + ContextPtr context_, + ObjectInfos * object_infos_) + : WithContext(context_), bucket(bucket_), query(query_), virtual_header(virtual_header_) + { + Strings all_keys = keys_; + + /// Create a virtual block with one row to construct filter + if (query && virtual_header && !all_keys.empty()) + { + /// Append "idx" column as the filter result + virtual_header.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + + auto block = virtual_header.cloneEmpty(); + addPathToVirtualColumns(block, fs::path(bucket) / all_keys.front(), 0); + + ASTPtr filter_ast; + VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast); + + if (filter_ast) + { + block = virtual_header.cloneEmpty(); + for (size_t i = 0; i < all_keys.size(); ++i) + addPathToVirtualColumns(block, fs::path(bucket) / all_keys[i], i); + + VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); + const auto & idxs = typeid_cast(*block.getByName("_idx").column); + + Strings filtered_keys; + filtered_keys.reserve(block.rows()); + for (UInt64 idx : idxs.getData()) + filtered_keys.emplace_back(std::move(all_keys[idx])); + + all_keys = std::move(filtered_keys); + } + } + + for (auto && key : all_keys) + { + std::optional info; + + /// To avoid extra requests update total_size only if object_infos != nullptr + /// (which means we eventually need this info anyway, so it should be ok to do it now) + if (object_infos_ != nullptr) + { + info = DB::S3::getObjectInfo(*client_, bucket, key, version_id_, request_settings_); + total_size += info->size; + + String path = fs::path(bucket) / key; + (*object_infos_)[std::move(path)] = *info; + } + + keys.emplace_back(std::move(key), std::move(info)); + } + } + + KeyWithInfo next() + { + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= keys.size()) + return {}; + + return keys[current_index]; + } + + size_t getTotalSize() const { return total_size; } + +private: + KeysWithInfo keys; + std::atomic_size_t index = 0; + + String bucket; + ASTPtr query; + Block virtual_header; + + size_t total_size = 0; +}; + +IcebergSource::KeysIterator::KeysIterator( + const std::shared_ptr & client_, /// proton: updated + const std::string & version_id_, + const std::vector & keys_, + const String & bucket_, + const S3Settings::RequestSettings & request_settings_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + ObjectInfos * object_infos) + : pimpl(std::make_shared( + client_, version_id_, keys_, bucket_, request_settings_, query, virtual_header, context, object_infos)) +{ +} + +IcebergSource::KeyWithInfo IcebergSource::KeysIterator::next() +{ + return pimpl->next(); +} + +size_t IcebergSource::KeysIterator::getTotalSize() const +{ + return pimpl->getTotalSize(); +} + +Block IcebergSource::getHeader(Block sample_block, const std::vector & requested_virtual_columns) +{ + for (const auto & virtual_column : requested_virtual_columns) + sample_block.insert({virtual_column.type->createColumn(), virtual_column.type, virtual_column.name}); + + return sample_block; +} + +IcebergSource::IcebergSource( + const std::vector & requested_virtual_columns_, + const String & format_, + String name_, + const Block & sample_block_, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + const String compression_hint_, + const std::shared_ptr & client_, + const String & bucket_, + const String & version_id_, + std::shared_ptr file_iterator_, + const size_t download_thread_num_) + : ISource(getHeader(sample_block_, requested_virtual_columns_), true, ProcessorID::IcebergSourceID) + , WithContext(context_) + , name(std::move(name_)) + , bucket(bucket_) + , version_id(version_id_) + , format(format_) + , columns_desc(columns_) + , max_block_size(max_block_size_) + , request_settings(request_settings_) + , compression_hint(compression_hint_) + , client(client_) + , sample_block(sample_block_) + , format_settings(format_settings_) + , requested_virtual_columns(requested_virtual_columns_) + , file_iterator(file_iterator_) + , download_thread_num(download_thread_num_) + , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, 1) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) +{ + reader = createReader(); + if (reader) + reader_future = createReaderAsync(); +} + + +void IcebergSource::onCancel() +{ + std::lock_guard lock(reader_mutex); + if (reader) + reader->cancel(); +} + + +IcebergSource::ReaderHolder IcebergSource::createReader() +{ + auto [current_key, info] = (*file_iterator)(); + if (current_key.empty()) + return {}; + + size_t object_size = info ? info->size : DB::S3::getObjectSize(*client, bucket, current_key, version_id, request_settings); + + auto compression_method = chooseCompressionMethod(current_key, compression_hint); + + InputFormatPtr input_format; + std::unique_ptr owned_read_buf; + + auto read_buf_or_factory = createS3ReadBuffer(current_key, object_size); + if (read_buf_or_factory.buf_factory) + { + input_format = FormatFactory::instance().getInputRandomAccess( + format, + std::move(read_buf_or_factory.buf_factory), + sample_block, + getContext(), + max_block_size, + /* is_remote_fs */ true, + compression_method, + format_settings); + } + else + { + owned_read_buf = wrapReadBufferWithCompressionMethod( + std::move(read_buf_or_factory.buf), compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + input_format + = FormatFactory::instance().getInput(format, *owned_read_buf, sample_block, getContext(), max_block_size, format_settings); + } + + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform([&](const Block & header) { + return std::make_shared(header, columns_desc, *input_format, getContext()); + }); + } + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + return ReaderHolder{fs::path(bucket) / current_key, std::move(owned_read_buf), std::move(pipeline), std::move(current_reader)}; +} + +std::future IcebergSource::createReaderAsync() +{ + return create_reader_scheduler([this] { return createReader(); }, Priority{}); +} + +IcebergSource::ReadBufferOrFactory IcebergSource::createS3ReadBuffer(const String & key, size_t object_size) +{ + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + const bool object_too_small = object_size <= 2 * download_buffer_size; + + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + { + LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmissing-field-initializers" + return {.buf = createAsyncS3ReadBuffer(key, read_settings, object_size)}; +#pragma clang diagnostic pop + } + + auto factory = std::make_unique(client, bucket, key, version_id, object_size, request_settings, read_settings); +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wmissing-field-initializers" + return {.buf_factory = std::move(factory)}; +#pragma clang diagnostic pop +} + +std::unique_ptr +IcebergSource::createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size) +{ + auto context = getContext(); + auto read_buffer_creator = [this, read_settings, object_size]( + const std::string & path, size_t read_until_position) -> std::unique_ptr { + return std::make_unique( + client, + bucket, + path, + version_id, + request_settings, + read_settings, + /* use_external_buffer */ true, + /* offset */ 0, + read_until_position, + /* restricted_seek */ true, + object_size); + }; + + auto s3_impl = std::make_unique( + std::move(read_buffer_creator), + StoredObjects{StoredObject{key, object_size}}, + read_settings, + /* cache_log */ nullptr); + + auto modified_settings{read_settings}; + /// FIXME: Changing this setting to default value breaks something around parquet reading + modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; + + auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + auto async_reader = std::make_unique( + std::move(s3_impl), pool_reader, modified_settings, context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; +} + +IcebergSource::~IcebergSource() +{ + create_reader_pool.wait(); +} + +String IcebergSource::getName() const +{ + return name; +} + +Chunk IcebergSource::generate() +{ + while (true) + { + if (!reader || isCancelled()) + break; + + Chunk chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + + const auto & file_path = reader.getPath(); + size_t total_size = file_iterator->getTotalSize(); + if ((num_rows != 0u) && (total_size != 0u)) + { + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + } + + for (const auto & virtual_column : requested_virtual_columns) + { + if (virtual_column.name == "_path") + { + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_file") + { + size_t last_slash_pos = file_path.find_last_of('/'); + auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + } + + return chunk; + } + + { + std::lock_guard lock(reader_mutex); + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool.wait(); + reader_future = createReaderAsync(); + } + } + return {}; +} + +} + +} +#endif diff --git a/src/Storages/ExternalStream/Iceberg/IcebergSource.h b/src/Storages/ExternalStream/Iceberg/IcebergSource.h new file mode 100644 index 00000000000..4dc623b6aea --- /dev/null +++ b/src/Storages/ExternalStream/Iceberg/IcebergSource.h @@ -0,0 +1,184 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ExternalStream +{ + +class IcebergSource final : public ISource, WithContext +{ +public: + struct KeyWithInfo + { + KeyWithInfo() = default; + KeyWithInfo(String key_, std::optional info_) : key(std::move(key_)), info(std::move(info_)) { } + + String key; + std::optional info; + }; + + using KeysWithInfo = std::vector; + using ObjectInfos = std::unordered_map; + class IIterator + { + public: + virtual ~IIterator() = default; + virtual KeyWithInfo next() = 0; + virtual size_t getTotalSize() const = 0; + + KeyWithInfo operator()() { return next(); } + }; + + class KeysIterator : public IIterator + { + public: + explicit KeysIterator( + const std::shared_ptr & client_, /// proton: updated + const std::string & version_id_, + const std::vector & keys_, + const String & bucket_, + const S3Settings::RequestSettings & request_settings_, + ASTPtr query, + const Block & virtual_header, + ContextPtr context, + ObjectInfos * object_infos = nullptr); + + KeyWithInfo next() override; + size_t getTotalSize() const override; + + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + + class ReadTaskIterator : public IIterator + { + public: + explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) { } + + KeyWithInfo next() override { return {callback(), {}}; } + + size_t getTotalSize() const override { return 0; } + + private: + ReadTaskCallback callback; + }; + + static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); + + IcebergSource( + const std::vector & requested_virtual_columns_, + const String & format, + String name_, + const Block & sample_block, + ContextPtr context_, + std::optional format_settings_, + const ColumnsDescription & columns_, + UInt64 max_block_size_, + const S3Settings::RequestSettings & request_settings_, + String compression_hint_, + const std::shared_ptr & client_, + const String & bucket, + const String & version_id, + std::shared_ptr file_iterator_, + size_t download_thread_num); + + ~IcebergSource() override; + + String getName() const override; + + Chunk generate() override; + + void onCancel() override; + +private: + String name; + String bucket; + String version_id; + String format; + ColumnsDescription columns_desc; + UInt64 max_block_size; + S3Settings::RequestSettings request_settings; + String compression_hint; + std::shared_ptr client; + Block sample_block; + std::optional format_settings; + + struct ReaderHolder + { + public: + ReaderHolder( + String path_, + std::unique_ptr read_buf_, + std::unique_ptr pipeline_, + std::unique_ptr reader_) + : path(std::move(path_)), read_buf(std::move(read_buf_)), pipeline(std::move(pipeline_)), reader(std::move(reader_)) + { + } + + ReaderHolder() = default; + + explicit operator bool() const { return reader != nullptr; } + PullingPipelineExecutor * operator->() { return reader.get(); } + const PullingPipelineExecutor * operator->() const { return reader.get(); } + const String & getPath() const { return path; } + + private: + String path; + std::unique_ptr read_buf; + std::unique_ptr pipeline; + std::unique_ptr reader; + }; + + struct ReadBufferOrFactory + { + std::unique_ptr buf; + SeekableReadBufferFactoryPtr buf_factory; + }; + + ReaderHolder reader; + + /// onCancel and generate can be called concurrently + std::mutex reader_mutex; + std::vector requested_virtual_columns; + std::shared_ptr file_iterator; + size_t download_thread_num = 1; + + Poco::Logger * log = &Poco::Logger::get("IcebergSource"); + + ThreadPool create_reader_pool; + ThreadPoolCallbackRunner create_reader_scheduler; + std::future reader_future; + + UInt64 total_rows_approx_max = 0; + size_t total_rows_count_times = 0; + UInt64 total_rows_approx_accumulated = 0; + + /// Recreate ReadBuffer and BlockInputStream for each file. + ReaderHolder createReader(); + std::future createReaderAsync(); + + ReadBufferOrFactory createS3ReadBuffer(const String & key, size_t object_size); + std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); +}; + +} + +} + +#endif diff --git a/src/Storages/Iceberg/AvroSchemas.h b/src/Storages/Iceberg/AvroSchemas.h new file mode 100644 index 00000000000..c06db54fd9b --- /dev/null +++ b/src/Storages/Iceberg/AvroSchemas.h @@ -0,0 +1,25 @@ +#pragma once + +namespace Apache::Iceberg +{ + +namespace AvroSchemas +{ + +/// These schemas were generated from the manifest list and manifest files generated by pyiceberg library, with avro-tools. +static const std::string MANIFEST_LIST + = R"({"type":"record","name":"manifest_file","fields":[{"name":"manifest_path","type":"string","doc":"Location URI with FS scheme","field-id":500},{"name":"manifest_length","type":"long","doc":"Total file size in bytes","field-id":501},{"name":"partition_spec_id","type":"int","doc":"Spec ID used to write","field-id":502},{"name":"content","type":"int","doc":"Contents of the manifest: 0=data, 1=deletes","field-id":517},{"name":"sequence_number","type":"long","doc":"Sequence number when the manifest was added","field-id":515},{"name":"min_sequence_number","type":"long","doc":"Lowest sequence number in the manifest","field-id":516},{"name":"added_snapshot_id","type":"long","doc":"Snapshot ID that added the manifest","field-id":503},{"name":"added_files_count","type":"int","doc":"Added entry count","field-id":504},{"name":"existing_files_count","type":"int","doc":"Existing entry count","field-id":505},{"name":"deleted_files_count","type":"int","doc":"Deleted entry count","field-id":506},{"name":"added_rows_count","type":"long","doc":"Added rows count","field-id":512},{"name":"existing_rows_count","type":"long","doc":"Existing rows count","field-id":513},{"name":"deleted_rows_count","type":"long","doc":"Deleted rows count","field-id":514},{"name":"partitions","type":["null",{"type":"array","items":{"type":"record","name":"r508","fields":[{"name":"contains_null","type":"boolean","doc":"True if any file has a null partition value","field-id":509},{"name":"contains_nan","type":["null","boolean"],"doc":"True if any file has a nan partition value","default":null,"field-id":518},{"name":"lower_bound","type":["null","bytes"],"doc":"Partition lower bound for all files","default":null,"field-id":510},{"name":"upper_bound","type":["null","bytes"],"doc":"Partition upper bound for all files","default":null,"field-id":511}]},"element-id":508}],"doc":"Summary for each partition","default":null,"field-id":507},{"name":"key_metadata","type":["null","bytes"],"doc":"Encryption key metadata blob","default":null,"field-id":519}]})"; + +/// Same manifest list schema, but in Iceberg schema format. +static const std::string MANIFEST_LIST_ICEBERG + = R"({"type":"struct","schema-id":0,"fields":[{"id":500,"name":"manifest_path","required":true,"type":"string","doc":"Location URI with FS scheme"},{"id":501,"name":"manifest_length","required":true,"type":"long","doc":"Total file size in bytes"},{"id":502,"name":"partition_spec_id","required":true,"type":"int","doc":"Spec ID used to write"},{"id":517,"name":"content","required":true,"type":"int","doc":"Contents of the manifest: 0=data, 1=deletes"},{"id":515,"name":"sequence_number","required":true,"type":"long","doc":"Sequence number when the manifest was added"},{"id":516,"name":"min_sequence_number","required":true,"type":"long","doc":"Lowest sequence number in the manifest"},{"id":503,"name":"added_snapshot_id","required":true,"type":"long","doc":"Snapshot ID that added the manifest"},{"id":504,"name":"added_files_count","required":true,"type":"int","doc":"Added entry count"},{"id":505,"name":"existing_files_count","required":true,"type":"int","doc":"Existing entry count"},{"id":506,"name":"deleted_files_count","required":true,"type":"int","doc":"Deleted entry count"},{"id":512,"name":"added_rows_count","required":true,"type":"long","doc":"Added rows count"},{"id":513,"name":"existing_rows_count","required":true,"type":"long","doc":"Existing rows count"},{"id":514,"name":"deleted_rows_count","required":true,"type":"long","doc":"Deleted rows count"},{"id":507,"name":"partitions","required":false,"type":{"type":"list","element-id":508,"element":{"type":"struct","fields":[{"id":509,"name":"contains_null","required":true,"type":"boolean","doc":"True if any file has a null partition value"},{"id":518,"name":"contains_nan","required":false,"type":"boolean","doc":"True if any file has a nan partition value"},{"id":510,"name":"lower_bound","required":false,"type":"binary","doc":"Partition lower bound for all files"},{"id":511,"name":"upper_bound","required":false,"type":"binary","doc":"Partition upper bound for all files"}]},"element-required":true},"doc":"Summary for each partition"},{"id":519,"name":"key_metadata","required":false,"type":"binary","doc":"Encryption key metadata blob"}]})"; + +static const std::string MANIFEST + = R"({"type":"record","name":"manifest_entry","fields":[{"name":"status","type":"int","field-id":0},{"name":"snapshot_id","type":["null","long"],"default":null,"field-id":1},{"name":"sequence_number","type":["null","long"],"default":null,"field-id":3},{"name":"file_sequence_number","type":["null","long"],"default":null,"field-id":4},{"name":"data_file","type":{"type":"record","name":"r2","fields":[{"name":"content","type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes","field-id":134},{"name":"file_path","type":"string","doc":"Location URI with FS scheme","field-id":100},{"name":"file_format","type":"string","doc":"File format name: avro, orc, or parquet","field-id":101},{"name":"partition","type":{"type":"record","name":"r102","fields":[]},"doc":"Partition data tuple, schema based on the partition spec","field-id":102},{"name":"record_count","type":"long","doc":"Number of records in the file","field-id":103},{"name":"file_size_in_bytes","type":"long","doc":"Total file size in bytes","field-id":104},{"name":"column_sizes","type":["null",{"type":"array","items":{"type":"record","name":"k117_v118","fields":[{"name":"key","type":"int","field-id":117},{"name":"value","type":"long","field-id":118}]},"logicalType":"map"}],"doc":"Map of column id to total size on disk","default":null,"field-id":108},{"name":"value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k119_v120","fields":[{"name":"key","type":"int","field-id":119},{"name":"value","type":"long","field-id":120}]},"logicalType":"map"}],"doc":"Map of column id to total count, including null and NaN","default":null,"field-id":109},{"name":"null_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k121_v122","fields":[{"name":"key","type":"int","field-id":121},{"name":"value","type":"long","field-id":122}]},"logicalType":"map"}],"doc":"Map of column id to null value count","default":null,"field-id":110},{"name":"nan_value_counts","type":["null",{"type":"array","items":{"type":"record","name":"k138_v139","fields":[{"name":"key","type":"int","field-id":138},{"name":"value","type":"long","field-id":139}]},"logicalType":"map"}],"doc":"Map of column id to number of NaN values in the column","default":null,"field-id":137},{"name":"lower_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k126_v127","fields":[{"name":"key","type":"int","field-id":126},{"name":"value","type":"bytes","field-id":127}]},"logicalType":"map"}],"doc":"Map of column id to lower bound","default":null,"field-id":125},{"name":"upper_bounds","type":["null",{"type":"array","items":{"type":"record","name":"k129_v130","fields":[{"name":"key","type":"int","field-id":129},{"name":"value","type":"bytes","field-id":130}]},"logicalType":"map"}],"doc":"Map of column id to upper bound","default":null,"field-id":128},{"name":"key_metadata","type":["null","bytes"],"doc":"Encryption key metadata blob","default":null,"field-id":131},{"name":"split_offsets","type":["null",{"type":"array","items":"long","element-id":133}],"doc":"Splittable offsets","default":null,"field-id":132},{"name":"equality_ids","type":["null",{"type":"array","items":"int","element-id":136}],"doc":"Equality comparison field IDs","default":null,"field-id":135},{"name":"sort_order_id","type":["null","int"],"doc":"Sort order ID","default":null,"field-id":140}]},"field-id":2}]})"; + +/// Same manifest schema, but in Iceberg schema format. +static const std::string MANIFEST_ICEBERG + = R"({"type":"struct","schema-id":0,"fields":[{"id":0,"name":"status","required":true,"type":"int"},{"id":1,"name":"snapshot_id","required":false,"type":"long"},{"id":3,"name":"sequence_number","required":false,"type":"long"},{"id":4,"name":"file_sequence_number","required":false,"type":"long"},{"id":2,"name":"data_file","required":true,"type":{"type":"struct","fields":[{"id":134,"name":"content","required":true,"type":"int","doc":"Contents of the file: 0=data, 1=position deletes, 2=equality deletes"},{"id":100,"name":"file_path","required":true,"type":"string","doc":"Location URI with FS scheme"},{"id":101,"name":"file_format","required":true,"type":"string","doc":"File format name: avro, orc, or parquet"},{"id":102,"name":"partition","required":true,"type":{"type":"struct","fields":[]},"doc":"Partition data tuple, schema based on the partition spec"},{"id":103,"name":"record_count","required":true,"type":"long","doc":"Number of records in the file"},{"id":104,"name":"file_size_in_bytes","required":true,"type":"long","doc":"Total file size in bytes"},{"id":108,"name":"column_sizes","required":false,"type":{"type":"map","key-id":117,"key":"int","value-id":118,"value":"long","value-required":true},"doc":"Map of column id to total size on disk"},{"id":109,"name":"value_counts","required":false,"type":{"type":"map","key-id":119,"key":"int","value-id":120,"value":"long","value-required":true},"doc":"Map of column id to total count, including null and NaN"},{"id":110,"name":"null_value_counts","required":false,"type":{"type":"map","key-id":121,"key":"int","value-id":122,"value":"long","value-required":true},"doc":"Map of column id to null value count"},{"id":137,"name":"nan_value_counts","required":false,"type":{"type":"map","key-id":138,"key":"int","value-id":139,"value":"long","value-required":true},"doc":"Map of column id to number of NaN values in the column"},{"id":125,"name":"lower_bounds","required":false,"type":{"type":"map","key-id":126,"key":"int","value-id":127,"value":"binary","value-required":true},"doc":"Map of column id to lower bound"},{"id":128,"name":"upper_bounds","required":false,"type":{"type":"map","key-id":129,"key":"int","value-id":130,"value":"binary","value-required":true},"doc":"Map of column id to upper bound"},{"id":131,"name":"key_metadata","required":false,"type":"binary","doc":"Encryption key metadata blob"},{"id":132,"name":"split_offsets","required":false,"type":{"type":"list","element-id":133,"element":"long","element-required":true},"doc":"Splittable offsets"},{"id":135,"name":"equality_ids","required":false,"type":{"type":"list","element-id":136,"element":"int","element-required":true},"doc":"Equality comparison field IDs"},{"id":140,"name":"sort_order_id","required":false,"type":"int","doc":"Sort order ID"}]}}]})"; +} + +} diff --git a/src/Storages/Iceberg/ICatalog.cpp b/src/Storages/Iceberg/ICatalog.cpp new file mode 100644 index 00000000000..2fd02f85436 --- /dev/null +++ b/src/Storages/Iceberg/ICatalog.cpp @@ -0,0 +1,132 @@ +#include + +#include +#include + +#include + +#include + +namespace DB::ErrorCodes +{ +extern const int NOT_IMPLEMENTED; +extern const int LOGICAL_ERROR; +} + +namespace Apache::Iceberg +{ + +StorageType parseStorageTypeFromLocation(const std::string & location) +{ + /// Table location in catalog metadata always starts with one of s3://, file://, etc. + /// So just extract this part of the path and deduce storage type from it. + + auto pos = location.find("://"); + if (pos == std::string::npos) + { + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unexpected path format: {}", location); + } + + auto storage_type_str = location.substr(0, pos); + auto storage_type = magic_enum::enum_cast(Poco::toUpper(storage_type_str)); + + if (!storage_type) + { + throw DB::Exception(DB::ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_str); + } + + return *storage_type; +} + +void TableMetadata::setLocation(const std::string & location_) +{ + if (!with_location) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data location was not requested"); + + /// 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(&Poco::Logger::get("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 + 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_; +} + +void TableMetadata::setSchemaJSON(const std::string & schema_json_) +{ + if (!with_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + iceberg_schema = schema_json_; +} + +const DB::NamesAndTypesList & TableMetadata::getSchema() const +{ + if (!with_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + return schema; +} + +const std::string & TableMetadata::getSchemaJSON() const +{ + if (!with_schema) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + return iceberg_schema; +} + +void TableMetadata::setStorageCredentials(std::shared_ptr credentials_) +{ + if (!with_storage_credentials) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Storage credentials were not requested"); + + storage_credentials = std::move(credentials_); +} + +std::shared_ptr TableMetadata::getStorageCredentials() const +{ + if (!with_storage_credentials) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Data schema was not requested"); + + return storage_credentials; +} + +StorageType TableMetadata::getStorageType() const +{ + return parseStorageTypeFromLocation(location_without_path); +} + +} diff --git a/src/Storages/Iceberg/ICatalog.h b/src/Storages/Iceberg/ICatalog.h new file mode 100644 index 00000000000..ff97f4c1ed0 --- /dev/null +++ b/src/Storages/Iceberg/ICatalog.h @@ -0,0 +1,274 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace Apache::Iceberg +{ +using StorageType = DB::ApacheIcebergStorageType; +StorageType parseStorageTypeFromLocation(const std::string & location); + +/// A class representing table metadata, +/// which was received from Catalog. +/* +{ + "config": { + "metadata_location": "s3://tp-gimi-test/hello_world/metadata/00001-9aebf755-36b2-4838-86c5-ff550dab047c.metadata.json", + "metadata_hashcode": "4daa63078f8baa111e03f08ec4c68b74d73fede391c5c689f76362e4893c357b", + "iceberg.table.uuid": "95711216-532a-441d-8973-11dcfb96d3c1", + "iceberg.table.lastUpdatedMs": "1739341126355", + "write.parquet.compression-codec": "zstd", + "table_type": "iceberg" + }, + "metadata": { + "current-schema-id": 0, + "current-snapshot-id": 704111412443037022, + "default-sort-order-id": 0, + "default-spec-id": 0, + "format-version": 2, + "last-column-id": 3, + "last-partition-id": 999, + "last-sequence-number": 1, + "last-updated-ms": 1739341126355, + "location": "s3://tp-gimi-test/hello_world", + "metadata-log": [], + "partition-specs": [ + { + "fields": [], + "spec-id": 0 + } + ], + "partition-statistics-files": [], + "properties": { + "write.parquet.compression-codec": "zstd" + }, + "refs": { + "main": { + "snapshot-id": 704111412443037022, + "type": "branch" + } + }, + "schemas": [ + { + "fields": [ + { + "id": 1, + "name": "id", + "required": true, + "type": "int" + }, + { + "id": 2, + "name": "key", + "required": true, + "type": "string" + }, + { + "id": 3, + "name": "value", + "required": true, + "type": "double" + } + ], + "schema-id": 0, + "type": "struct" + } + ], + "snapshot-log": [ + { + "snapshot-id": 704111412443037022, + "timestamp-ms": 1739341126355 + } + ], + "snapshots": [ + { + "manifest-list": "s3://tp-gimi-test/hello_world/metadata/snap-704111412443037022-1-00d110f2-5cbe-4802-a2d4-a2e1e8b0cdaf.avro", + "schema-id": 0, + "sequence-number": 1, + "snapshot-id": 704111412443037022, + "summary": { + "engine-version": "3.5.4", + "added-data-files": "1", + "total-equality-deletes": "0", + "app-id": "local-1739341024141", + "added-records": "1", + "total-records": "1", + "spark.app.id": "local-1739341024141", + "changed-partition-count": "1", + "engine-name": "spark", + "total-position-deletes": "0", + "added-files-size": "843", + "total-delete-files": "0", + "iceberg-version": "Apache Iceberg 1.7.1 (commit 4a432839233f2343a9eae8255532f911f06358ef)", + "total-files-size": "843", + "total-data-files": "1", + "operation": "append" + }, + "timestamp-ms": 1739341126355 + } + ], + "sort-orders": [ + { + "fields": [], + "order-id": 0 + } + ], + "statistics-files": [], + "table-uuid": "95711216-532a-441d-8973-11dcfb96d3c1" + }, + "metadata-location": "s3://tp-gimi-test/hello_world/metadata/00001-9aebf755-36b2-4838-86c5-ff550dab047c.metadata.json" +} +*/ +class TableMetadata +{ +public: + TableMetadata() = default; + + TableMetadata & withLocation() + { + with_location = true; + return *this; + } + TableMetadata & withSchema() + { + with_schema = true; + return *this; + } + TableMetadata & withStorageCredentials() + { + with_storage_credentials = true; + return *this; + } + + void setLocation(const std::string & location_); + std::string getLocation(bool path_only) const; + + void setSchema(const DB::NamesAndTypesList & schema_); + const DB::NamesAndTypesList & getSchema() const; + + void setSchemaJSON(const std::string & schema_json_); + const std::string & getSchemaJSON() const; + + void setStorageCredentials(std::shared_ptr credentials_); + std::shared_ptr getStorageCredentials() const; + + int64_t getSnapshotID() const { return snapshot_id; } + void setSnapshotID(int64_t id) { snapshot_id = id; } + + uint64_t getSequenceNumber() const { return sequence_number; } + void setSequenceNumber(uint64_t sn) { sequence_number = sn; } + + const std::string & getTableUUID() const { return table_uuid; } + void setTableUUID(std::string uuid) { table_uuid = uuid; } + + int getFormatVersion() const { return format_version; } + void setFormatVersion(int version) { format_version = version; } + + const std::string & getManifestList() const { return current_snapshot.manifest_list; } + + const Snapshot & getCurrentSnapshot() const { return current_snapshot; } + void setCurrentSnapshot(Snapshot && snapshot_) { current_snapshot = std::move(snapshot_); } + + bool requiresLocation() const { return with_location; } + bool requiresSchema() const { return with_schema; } + bool requiresCredentials() const { return with_storage_credentials; } + + StorageType getStorageType() const; + +private: + /// Starts with s3://, file://, etc. + /// For example, `s3://bucket/` + std::string location_without_path; + /// Path to table's data: `/path/to/table/data/` + std::string path; + DB::NamesAndTypesList schema; + std::string iceberg_schema; + + int64_t snapshot_id{0}; + uint64_t sequence_number{0}; + + std::string table_uuid; + int format_version{0}; + Snapshot current_snapshot; + + /// Storage credentials, which are called "vended credentials". + std::shared_ptr storage_credentials; + + bool with_location = false; + bool with_schema = false; + bool with_storage_credentials = false; +}; + + +/// Base class for catalog implementation. +/// Used for communication with the catalog. +class ICatalog +{ +public: + using Namespaces = std::vector; + + explicit ICatalog(const std::string & warehouse_) : warehouse(warehouse_) { } + + virtual ~ICatalog() = default; + + virtual bool existsNamespace(const std::string & name) const = 0; + virtual void createNamespace(const std::string & name) const = 0; + + /// Does catalog namespace have any tables? + virtual bool empty(const std::string & namespace_name) const = 0; + + /// Fetch tables' names list. + /// Contains full namespaces in names. + virtual DB::Names getTables(const std::string & namespace_name) const = 0; + + /// Check that a table exists in a given namespace. + virtual bool existsTable(const std::string & namespace_name, const std::string & table_name) const = 0; + + virtual void createTable( + const std::string & namespace_name, + const std::string & name, + std::optional location, + const DB::NamesAndTypesList & schema, /// TODO define a Schema struct/class. + std::optional partition_spec, + std::optional write_order, + bool stage_create, + std::unordered_map properties) + = 0; + + virtual void deleteTable(const std::string & namespace_name, const std::string & name) = 0; + + /// Commits the updates to the target table, and update table_metadata. + virtual void commitTable( + const std::string & namespace_name, + const std::string & name, + const Requirements & requirements, + const Updates & updates, + TableMetadata & table_metadata) + = 0; + + /// Get table metadata in the given namespace. + /// Throw exception if table does not exist. + virtual void getTableMetadata(const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const = 0; + + /// Get table metadata in the given namespace. + /// Return `false` if table does not exist, `true` otherwise. + virtual bool tryGetTableMetadata(const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const = 0; + + /// Get storage type, where Iceberg tables' data is stored. + /// E.g. one of S3, Azure, Local, HDFS. + virtual std::optional getStorageType() const = 0; + +protected: + /// Name of the warehouse, + /// which is sometimes also called "catalog name". + const std::string warehouse; +}; + +using CatalogPtr = std::shared_ptr; + +} diff --git a/src/Storages/Iceberg/IcebergMetadata.cpp b/src/Storages/Iceberg/IcebergMetadata.cpp new file mode 100644 index 00000000000..5934ae300c0 --- /dev/null +++ b/src/Storages/Iceberg/IcebergMetadata.cpp @@ -0,0 +1,122 @@ +#include + +#if USE_AVRO +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int FILE_DOESNT_EXIST; +extern const int ILLEGAL_COLUMN; +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +} + +std::pair parseTableSchemaV1Method(const Poco::JSON::Object::Ptr & metadata_object) +{ + if (!metadata_object->has("schema")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema' field is missing in metadata"); + Poco::JSON::Object::Ptr schema = metadata_object->getObject("schema"); + if (!schema->has("schema-id")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema-id' field is missing in schema"); + auto current_schema_id = schema->getValue("schema-id"); + return {schema, current_schema_id}; +} + +std::pair parseTableSchemaV2Method(const Poco::JSON::Object::Ptr & metadata_object) +{ + Poco::JSON::Object::Ptr schema; + if (!metadata_object->has("current-schema-id")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'current-schema-id' field is missing in metadata"); + auto current_schema_id = metadata_object->getValue("current-schema-id"); + if (!metadata_object->has("schemas")) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schemas' field is missing in metadata"); + auto schemas = metadata_object->get("schemas").extract(); + if (schemas->size() == 0) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: schemas field is empty"); + for (uint32_t i = 0; i != schemas->size(); ++i) + { + auto current_schema = schemas->getObject(i); + if (!current_schema->has("schema-id")) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse Iceberg table schema: 'schema-id' field is missing in schema"); + } + if (current_schema->getValue("schema-id") == current_schema_id) + { + schema = current_schema; + break; + } + } + + if (!schema) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(There is no schema with "schema-id" that matches "current-schema-id" in metadata)"); + if (schema->getValue("schema-id") != current_schema_id) + throw Exception(ErrorCodes::BAD_ARGUMENTS, R"(Field "schema-id" of the schema doesn't match "current-schema-id" in metadata)"); + return {schema, current_schema_id}; +} + +Int32 IcebergMetadata::parseTableSchema( + const Poco::JSON::Object::Ptr & metadata_object, + IcebergSchemaProcessor & schema_processor, + std::string & iceberg_schema_json, + Poco::Logger * metadata_logger) +{ + Int32 format_version = metadata_object->getValue("format-version"); + if (format_version == 2) + { + auto [schema, current_schema_id] = parseTableSchemaV2Method(metadata_object); + schema_processor.addIcebergTableSchema(schema); + { + std::stringstream ss; + schema->stringify(ss); + iceberg_schema_json = ss.str(); + } + + return current_schema_id; + } + else + { + try + { + auto [schema, current_schema_id] = parseTableSchemaV1Method(metadata_object); + schema_processor.addIcebergTableSchema(schema); + { + std::stringstream ss; + schema->stringify(ss); + iceberg_schema_json = ss.str(); + } + return current_schema_id; + } + catch (const Exception & first_error) + { + if (first_error.code() != ErrorCodes::BAD_ARGUMENTS) + throw; + try + { + auto [schema, current_schema_id] = parseTableSchemaV2Method(metadata_object); + schema_processor.addIcebergTableSchema(schema); + LOG_WARNING( + metadata_logger, + "Iceberg table schema was parsed using v2 specification, but it was impossible to parse it using v1 " + "specification. Be " + "aware that you Iceberg writing engine violates Iceberg specification. Error during parsing {}", + first_error.displayText()); + return current_schema_id; + } + catch (const Exception & second_error) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot parse Iceberg table schema both with v1 and v2 methods. V1 parsing error: {}. V2 parsing error: {}", + first_error.displayText(), + second_error.displayText()); + } + } + } +} + +} + +#endif diff --git a/src/Storages/Iceberg/IcebergMetadata.h b/src/Storages/Iceberg/IcebergMetadata.h new file mode 100644 index 00000000000..e448ed02246 --- /dev/null +++ b/src/Storages/Iceberg/IcebergMetadata.h @@ -0,0 +1,24 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO +#include + +#include + +namespace DB +{ + +class IcebergMetadata +{ +public: + static Int32 parseTableSchema( + const Poco::JSON::Object::Ptr & metadata_object, + IcebergSchemaProcessor & schema_processor, + std::string & iceberg_schema_json, + Poco::Logger * metadata_logger); +}; + +} +#endif diff --git a/src/Storages/Iceberg/Manifest.h b/src/Storages/Iceberg/Manifest.h new file mode 100644 index 00000000000..0e75c83ee68 --- /dev/null +++ b/src/Storages/Iceberg/Manifest.h @@ -0,0 +1,1514 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* This code was generated by avrogencpp 1.11.3. Do not edit.*/ + +/* proton: this code was modiffied for fixing all the compiler warnings and matching our code style. */ + +#pragma once + +#include +#include +#include + +#include + +namespace Apache::Iceberg +{ +struct SnapshotID +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + int64_t get_long() const; + void set_long(const int64_t & v); + + SnapshotID(); +}; + +struct SequenceNumber +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + int64_t get_long() const; + void set_long(const int64_t & v); + + SequenceNumber(); +}; + +struct FileSequenceNumber +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + int64_t get_long() const; + void set_long(const int64_t & v); + FileSequenceNumber(); +}; + +struct Partition +{ + Partition() = default; +}; + +struct ColumnSize +{ + int32_t key; + int64_t value; + ColumnSize() : key(int32_t()), value(int64_t()) { } +}; + +struct ColumnSizes +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + ColumnSizes(); +}; + +struct ValueCount +{ + int32_t key; + int64_t value; + ValueCount() : key(int32_t()), value(int64_t()) { } +}; + +struct ValueCounts +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + ValueCounts(); +}; + +struct NullValueCount +{ + int32_t key; + int64_t value; + NullValueCount() : key(int32_t()), value(int64_t()) { } +}; + +struct NullValueCounts +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + NullValueCounts(); +}; + +struct NanValueCount +{ + int32_t key; + int64_t value; + NanValueCount() : key(int32_t()), value(int64_t()) { } +}; + +struct NanValueCounts +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + NanValueCounts(); +}; + +struct LowerBound +{ + int32_t key; + std::vector value; + LowerBound() : key(int32_t()) { } +}; + +struct LowerBounds +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + LowerBounds(); +}; + +struct UpperBound +{ + int32_t key; + std::vector value; + UpperBound() : key(int32_t()) { } +}; + +struct UpperBounds +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + UpperBounds(); +}; + +struct KeyMetadata +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_bytes() const; + void set_bytes(const std::vector & v); + KeyMetadata(); +}; + +struct SplitOffsets +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + SplitOffsets(); +}; + +struct EqualityIds +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + EqualityIds(); +}; + +struct SortOrderId +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + int32_t get_int() const; + void set_int(const int32_t & v); + SortOrderId(); +}; + +struct DataFile +{ + int32_t content; + std::string file_path; + std::string file_format; + Partition partition; + int64_t record_count; + int64_t file_size_in_bytes; + ColumnSizes column_sizes; + ValueCounts value_counts; + NullValueCounts null_value_counts; + NanValueCounts nan_value_counts; + LowerBounds lower_bounds; + UpperBounds upper_bounds; + KeyMetadata key_metadata; + SplitOffsets split_offsets; + EqualityIds equality_ids; + SortOrderId sort_order_id; + DataFile() : content(int32_t()), record_count(int64_t()), file_size_in_bytes(int64_t()) { } +}; + +struct Manifest +{ + int32_t status; + SnapshotID snapshot_id; + SequenceNumber sequence_number; + FileSequenceNumber file_sequence_number; + DataFile data_file; + Manifest() : status(int32_t()) { } +}; + +inline int64_t SnapshotID::get_long() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__0__"); + } + return std::any_cast(value_); +} + +inline void SnapshotID::set_long(const int64_t & v) +{ + idx_ = 1; + value_ = v; +} + +inline int64_t SequenceNumber::get_long() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__1__"); + } + return std::any_cast(value_); +} + +inline void SequenceNumber::set_long(const int64_t & v) +{ + idx_ = 1; + value_ = v; +} + +inline int64_t FileSequenceNumber::get_long() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__2__"); + } + return std::any_cast(value_); +} + +inline void FileSequenceNumber::set_long(const int64_t & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector ColumnSizes::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__3__"); + } + return std::any_cast>(value_); +} + +inline void ColumnSizes::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector ValueCounts::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__4__"); + } + return std::any_cast>(value_); +} + +inline void ValueCounts::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector NullValueCounts::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__5__"); + } + return std::any_cast>(value_); +} + +inline void NullValueCounts::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector NanValueCounts::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__6__"); + } + return std::any_cast>(value_); +} + +inline void NanValueCounts::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector LowerBounds::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__7__"); + } + return std::any_cast>(value_); +} + +inline void LowerBounds::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector UpperBounds::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__8__"); + } + return std::any_cast>(value_); +} + +inline void UpperBounds::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector KeyMetadata::get_bytes() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__9__"); + } + return std::any_cast>(value_); +} + +inline void KeyMetadata::set_bytes(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector SplitOffsets::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__10__"); + } + return std::any_cast>(value_); +} + +inline void SplitOffsets::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector EqualityIds::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__11__"); + } + return std::any_cast>(value_); +} + +inline void EqualityIds::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline int32_t SortOrderId::get_int() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union cpx_json_Union__12__"); + } + return std::any_cast(value_); +} + +inline void SortOrderId::set_int(const int32_t & v) +{ + idx_ = 1; + value_ = v; +} + +inline SnapshotID::SnapshotID() : idx_(0) +{ +} +inline SequenceNumber::SequenceNumber() : idx_(0) +{ +} +inline FileSequenceNumber::FileSequenceNumber() : idx_(0) +{ +} +inline ColumnSizes::ColumnSizes() : idx_(0) +{ +} +inline ValueCounts::ValueCounts() : idx_(0) +{ +} +inline NullValueCounts::NullValueCounts() : idx_(0) +{ +} +inline NanValueCounts::NanValueCounts() : idx_(0) +{ +} +inline LowerBounds::LowerBounds() : idx_(0) +{ +} +inline UpperBounds::UpperBounds() : idx_(0) +{ +} +inline KeyMetadata::KeyMetadata() : idx_(0) +{ +} +inline SplitOffsets::SplitOffsets() : idx_(0) +{ +} +inline EqualityIds::EqualityIds() : idx_(0) +{ +} +inline SortOrderId::SortOrderId() : idx_(0) +{ +} +} +namespace avro +{ +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::SnapshotID v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_long()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::SnapshotID & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + int64_t vv; + avro::decode(d, vv); + v.set_long(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::SequenceNumber v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_long()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::SequenceNumber & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + int64_t vv; + avro::decode(d, vv); + v.set_long(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::FileSequenceNumber v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_long()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::FileSequenceNumber & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + int64_t vv; + avro::decode(d, vv); + v.set_long(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & /*e*/, const Apache::Iceberg::Partition & /*v*/) { } + static void decode(Decoder & d, Apache::Iceberg::Partition & /*v*/) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + default: + break; + } + } + } + else + { + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::ColumnSize & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::ColumnSize & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::ColumnSizes v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::ColumnSizes & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::ValueCount & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::ValueCount & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::ValueCounts v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::ValueCounts & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::NullValueCount & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::NullValueCount & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::NullValueCounts v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::NullValueCounts & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::NanValueCount & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::NanValueCount & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::NanValueCounts v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::NanValueCounts & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::LowerBound & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::LowerBound & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::LowerBounds v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::LowerBounds & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::UpperBound & v) + { + avro::encode(e, v.key); + avro::encode(e, v.value); + } + static void decode(Decoder & d, Apache::Iceberg::UpperBound & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.key); + break; + case 1: + avro::decode(d, v.value); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.key); + avro::decode(d, v.value); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::UpperBounds v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::UpperBounds & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::KeyMetadata v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_bytes()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::KeyMetadata & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_bytes(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::SplitOffsets v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::SplitOffsets & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::EqualityIds v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::EqualityIds & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::SortOrderId v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_int()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::SortOrderId & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + int32_t vv; + avro::decode(d, vv); + v.set_int(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::DataFile & v) + { + avro::encode(e, v.content); + avro::encode(e, v.file_path); + avro::encode(e, v.file_format); + avro::encode(e, v.partition); + avro::encode(e, v.record_count); + avro::encode(e, v.file_size_in_bytes); + avro::encode(e, v.column_sizes); + avro::encode(e, v.value_counts); + avro::encode(e, v.null_value_counts); + avro::encode(e, v.nan_value_counts); + avro::encode(e, v.lower_bounds); + avro::encode(e, v.upper_bounds); + avro::encode(e, v.key_metadata); + avro::encode(e, v.split_offsets); + avro::encode(e, v.equality_ids); + avro::encode(e, v.sort_order_id); + } + static void decode(Decoder & d, Apache::Iceberg::DataFile & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.content); + break; + case 1: + avro::decode(d, v.file_path); + break; + case 2: + avro::decode(d, v.file_format); + break; + case 3: + avro::decode(d, v.partition); + break; + case 4: + avro::decode(d, v.record_count); + break; + case 5: + avro::decode(d, v.file_size_in_bytes); + break; + case 6: + avro::decode(d, v.column_sizes); + break; + case 7: + avro::decode(d, v.value_counts); + break; + case 8: + avro::decode(d, v.null_value_counts); + break; + case 9: + avro::decode(d, v.nan_value_counts); + break; + case 10: + avro::decode(d, v.lower_bounds); + break; + case 11: + avro::decode(d, v.upper_bounds); + break; + case 12: + avro::decode(d, v.key_metadata); + break; + case 13: + avro::decode(d, v.split_offsets); + break; + case 14: + avro::decode(d, v.equality_ids); + break; + case 15: + avro::decode(d, v.sort_order_id); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.content); + avro::decode(d, v.file_path); + avro::decode(d, v.file_format); + avro::decode(d, v.partition); + avro::decode(d, v.record_count); + avro::decode(d, v.file_size_in_bytes); + avro::decode(d, v.column_sizes); + avro::decode(d, v.value_counts); + avro::decode(d, v.null_value_counts); + avro::decode(d, v.nan_value_counts); + avro::decode(d, v.lower_bounds); + avro::decode(d, v.upper_bounds); + avro::decode(d, v.key_metadata); + avro::decode(d, v.split_offsets); + avro::decode(d, v.equality_ids); + avro::decode(d, v.sort_order_id); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::Manifest & v) + { + avro::encode(e, v.status); + avro::encode(e, v.snapshot_id); + avro::encode(e, v.sequence_number); + avro::encode(e, v.file_sequence_number); + avro::encode(e, v.data_file); + } + static void decode(Decoder & d, Apache::Iceberg::Manifest & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.status); + break; + case 1: + avro::decode(d, v.snapshot_id); + break; + case 2: + avro::decode(d, v.sequence_number); + break; + case 3: + avro::decode(d, v.file_sequence_number); + break; + case 4: + avro::decode(d, v.data_file); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.status); + avro::decode(d, v.snapshot_id); + avro::decode(d, v.sequence_number); + avro::decode(d, v.file_sequence_number); + avro::decode(d, v.data_file); + } + } +}; + +} diff --git a/src/Storages/Iceberg/ManifestList.h b/src/Storages/Iceberg/ManifestList.h new file mode 100644 index 00000000000..f512b3d3d8a --- /dev/null +++ b/src/Storages/Iceberg/ManifestList.h @@ -0,0 +1,609 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* This code was generated by avrogencpp 1.11.3. Do not edit.*/ + +#pragma once + +#include +#include +#include + +#include + +namespace Apache::Iceberg +{ +struct r508_contains_nan +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + bool get_bool() const; + void set_bool(const bool & v); + r508_contains_nan(); +}; + +struct r508_lower_bound +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_bytes() const; + void set_bytes(const std::vector & v); + r508_lower_bound(); +}; + +struct r508_upper_bound +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_bytes() const; + void set_bytes(const std::vector & v); + r508_upper_bound(); +}; + +struct r508 +{ + bool contains_null; + r508_contains_nan contains_nan; + r508_lower_bound lower_bound; + r508_upper_bound upper_bound; + r508() : contains_null(bool()) { } +}; + +struct partitions +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_array() const; + void set_array(const std::vector & v); + partitions(); +}; + +struct key_metadata +{ +private: + size_t idx_; + std::any value_; + +public: + size_t idx() const { return idx_; } + bool is_null() const { return (idx_ == 0); } + void set_null() + { + idx_ = 0; + value_ = std::any(); + } + std::vector get_bytes() const; + void set_bytes(const std::vector & v); + key_metadata(); +}; + +struct ManifestList +{ + std::string manifest_path; + int64_t manifest_length; + int32_t partition_spec_id; + int32_t content; + int64_t sequence_number; + int64_t min_sequence_number; + int64_t added_snapshot_id; + int32_t added_files_count; + int32_t existing_files_count; + int32_t deleted_files_count; + int64_t added_rows_count; + int64_t existing_rows_count; + int64_t deleted_rows_count; + partitions partitions; + key_metadata key_metadata; + ManifestList() + : manifest_length(int64_t()) + , partition_spec_id(int32_t()) + , content(int32_t()) + , sequence_number(int64_t()) + , min_sequence_number(int64_t()) + , added_snapshot_id(int64_t()) + , added_files_count(int32_t()) + , existing_files_count(int32_t()) + , deleted_files_count(int32_t()) + , added_rows_count(int64_t()) + , existing_rows_count(int64_t()) + , deleted_rows_count(int64_t()) + { + } +}; + +inline bool r508_contains_nan::get_bool() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union r508_contains_nan"); + } + return std::any_cast(value_); +} + +inline void r508_contains_nan::set_bool(const bool & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector r508_lower_bound::get_bytes() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union r508_lower_bound"); + } + return std::any_cast>(value_); +} + +inline void r508_lower_bound::set_bytes(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector r508_upper_bound::get_bytes() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union r508_upper_bound"); + } + return std::any_cast>(value_); +} + +inline void r508_upper_bound::set_bytes(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector partitions::get_array() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union partitions"); + } + return std::any_cast>(value_); +} + +inline void partitions::set_array(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline std::vector key_metadata::get_bytes() const +{ + if (idx_ != 1) + { + throw avro::Exception("Invalid type for union key_metadata"); + } + return std::any_cast>(value_); +} + +inline void key_metadata::set_bytes(const std::vector & v) +{ + idx_ = 1; + value_ = v; +} + +inline r508_contains_nan::r508_contains_nan() : idx_(0) +{ +} +inline r508_lower_bound::r508_lower_bound() : idx_(0) +{ +} +inline r508_upper_bound::r508_upper_bound() : idx_(0) +{ +} +inline partitions::partitions() : idx_(0) +{ +} +inline key_metadata::key_metadata() : idx_(0) +{ +} +} +namespace avro +{ +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::r508_contains_nan v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_bool()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::r508_contains_nan & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + bool vv; + avro::decode(d, vv); + v.set_bool(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::r508_lower_bound v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_bytes()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::r508_lower_bound & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_bytes(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::r508_upper_bound v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_bytes()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::r508_upper_bound & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_bytes(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::r508 & v) + { + avro::encode(e, v.contains_null); + avro::encode(e, v.contains_nan); + avro::encode(e, v.lower_bound); + avro::encode(e, v.upper_bound); + } + static void decode(Decoder & d, Apache::Iceberg::r508 & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.contains_null); + break; + case 1: + avro::decode(d, v.contains_nan); + break; + case 2: + avro::decode(d, v.lower_bound); + break; + case 3: + avro::decode(d, v.upper_bound); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.contains_null); + avro::decode(d, v.contains_nan); + avro::decode(d, v.lower_bound); + avro::decode(d, v.upper_bound); + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::partitions v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_array()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::partitions & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_array(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, Apache::Iceberg::key_metadata v) + { + e.encodeUnionIndex(v.idx()); + switch (v.idx()) + { + case 0: + e.encodeNull(); + break; + case 1: + avro::encode(e, v.get_bytes()); + break; + } + } + static void decode(Decoder & d, Apache::Iceberg::key_metadata & v) + { + size_t n = d.decodeUnionIndex(); + if (n >= 2) + { + throw avro::Exception("Union index too big"); + } + switch (n) + { + case 0: + d.decodeNull(); + v.set_null(); + break; + case 1: + { + std::vector vv; + avro::decode(d, vv); + v.set_bytes(vv); + } + break; + } + } +}; + +template <> +struct codec_traits +{ + static void encode(Encoder & e, const Apache::Iceberg::ManifestList & v) + { + avro::encode(e, v.manifest_path); + avro::encode(e, v.manifest_length); + avro::encode(e, v.partition_spec_id); + avro::encode(e, v.content); + avro::encode(e, v.sequence_number); + avro::encode(e, v.min_sequence_number); + avro::encode(e, v.added_snapshot_id); + avro::encode(e, v.added_files_count); + avro::encode(e, v.existing_files_count); + avro::encode(e, v.deleted_files_count); + avro::encode(e, v.added_rows_count); + avro::encode(e, v.existing_rows_count); + avro::encode(e, v.deleted_rows_count); + avro::encode(e, v.partitions); + avro::encode(e, v.key_metadata); + } + static void decode(Decoder & d, Apache::Iceberg::ManifestList & v) + { + if (avro::ResolvingDecoder * rd = dynamic_cast(&d)) + { + const std::vector fo = rd->fieldOrder(); + for (auto it : fo) + { + switch (it) + { + case 0: + avro::decode(d, v.manifest_path); + break; + case 1: + avro::decode(d, v.manifest_length); + break; + case 2: + avro::decode(d, v.partition_spec_id); + break; + case 3: + avro::decode(d, v.content); + break; + case 4: + avro::decode(d, v.sequence_number); + break; + case 5: + avro::decode(d, v.min_sequence_number); + break; + case 6: + avro::decode(d, v.added_snapshot_id); + break; + case 7: + avro::decode(d, v.added_files_count); + break; + case 8: + avro::decode(d, v.existing_files_count); + break; + case 9: + avro::decode(d, v.deleted_files_count); + break; + case 10: + avro::decode(d, v.added_rows_count); + break; + case 11: + avro::decode(d, v.existing_rows_count); + break; + case 12: + avro::decode(d, v.deleted_rows_count); + break; + case 13: + avro::decode(d, v.partitions); + break; + case 14: + avro::decode(d, v.key_metadata); + break; + default: + break; + } + } + } + else + { + avro::decode(d, v.manifest_path); + avro::decode(d, v.manifest_length); + avro::decode(d, v.partition_spec_id); + avro::decode(d, v.content); + avro::decode(d, v.sequence_number); + avro::decode(d, v.min_sequence_number); + avro::decode(d, v.added_snapshot_id); + avro::decode(d, v.added_files_count); + avro::decode(d, v.existing_files_count); + avro::decode(d, v.deleted_files_count); + avro::decode(d, v.added_rows_count); + avro::decode(d, v.existing_rows_count); + avro::decode(d, v.deleted_rows_count); + avro::decode(d, v.partitions); + avro::decode(d, v.key_metadata); + } + } +}; + +} diff --git a/src/Storages/Iceberg/Requirement.cpp b/src/Storages/Iceberg/Requirement.cpp new file mode 100644 index 00000000000..3a92a271657 --- /dev/null +++ b/src/Storages/Iceberg/Requirement.cpp @@ -0,0 +1,33 @@ +#include + +#include +#include + +#include + +namespace Apache::Iceberg +{ + +void AssertRefSnapshotID::apply(Poco::JSON::Array & requirements) const +{ + assert(!ref.empty()); + + Poco::JSON::Object obj; + obj.set("type", "assert-ref-snapshot-id"); + obj.set("ref", ref); + if (snapshot_id > 0) + obj.set("snapshot-id", snapshot_id); + requirements.add(obj); +} + +void AssertTableUUID::apply(Poco::JSON::Array & requirements) const +{ + assert(!uuid.empty()); + + Poco::JSON::Object obj; + obj.set("type", "assert-table-uuid"); + obj.set("uuid", uuid); + requirements.add(obj); +} + +} diff --git a/src/Storages/Iceberg/Requirement.h b/src/Storages/Iceberg/Requirement.h new file mode 100644 index 00000000000..38899baec33 --- /dev/null +++ b/src/Storages/Iceberg/Requirement.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include + +namespace Poco::JSON +{ +class Array; +} + +namespace Apache::Iceberg +{ + +class Requirement +{ +public: + virtual ~Requirement() = default; + + virtual void apply(Poco::JSON::Array & requirements) const = 0; +}; + +class AssertRefSnapshotID : public Requirement +{ +public: + AssertRefSnapshotID(const std::string & ref_, int64_t snapshot_id_) : ref(ref_), snapshot_id(snapshot_id_) { } + ~AssertRefSnapshotID() override = default; + + void apply(Poco::JSON::Array & requirements) const override; + +private: + const std::string ref; + const int64_t snapshot_id{-1}; +}; + +class AssertTableUUID : public Requirement +{ +public: + explicit AssertTableUUID(const std::string & uuid_) : uuid(uuid_) { } + ~AssertTableUUID() override = default; + + void apply(Poco::JSON::Array & requirements) const override; + +private: + const std::string uuid; +}; + +using RequirementPtr = std::unique_ptr; +using Requirements = std::vector; + +} diff --git a/src/Storages/Iceberg/RestCatalog.cpp b/src/Storages/Iceberg/RestCatalog.cpp new file mode 100644 index 00000000000..0699130d638 --- /dev/null +++ b/src/Storages/Iceberg/RestCatalog.cpp @@ -0,0 +1,892 @@ +#include + +#if USE_AVRO +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int AWS_ERROR; +extern const int ICEBERG_CATALOG_ERROR; +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +} + +namespace CurrentMetrics +{ +extern const Metric LocalThread; +extern const Metric LocalThreadActive; +} + +namespace Apache::Iceberg +{ + +static constexpr auto CONFIG_ENDPOINT = "config"; +static constexpr auto NAMESPACES_ENDPOINT = "namespaces"; + +namespace +{ + +std::shared_ptr pickCredentialsProvider(const String & region) +{ + std::shared_ptr provider; + + auto client_config = DB::S3::ClientFactory::instance().createClientConfiguration( + /*force_region=*/region, + /*remote_host_filter=*/{}, + /*s3_max_redirects=*/10, + /*enable_s3_requests_logging=*/false, + /*for_disk_s3=*/false, + /*get_request_throttler=*/nullptr, + /*put_request_throttler=*/nullptr); + + DB::S3::S3CredentialsProviderChain provider_chain{ + /*configuration=*/client_config, + /*credentials=*/{}, + /*credentials_configuration=*/{.use_environment_credentials = true}}; + + for (size_t i = 0; const auto & cp : provider_chain.GetProviders()) + { + try + { + const auto creds = cp->GetAWSCredentials(); + if (!creds.IsEmpty()) + { + LOG_INFO(&Poco::Logger::get("AWS_MSK_IAM"), "The {}th provider from the provider chain has been picked", i); + provider = cp; + break; + } + ++i; + } + catch (...) + { + /// Just skip to the next one + } + } + + if (!provider) + throw DB::Exception(DB::ErrorCodes::AWS_ERROR, "No AWS credentials available"); + + return provider; +} + +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; + std::string client_secret; + 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); + } + 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 correctAPIURI(const std::string & uri) +{ + if (uri.ends_with("v1")) + return uri; + return std::filesystem::path(uri) / "v1"; +} + +std::string encodeNamespaceName(const std::string & namespace_name) +{ + std::string result; + result.reserve(namespace_name.size()); + + for (auto ch : namespace_name) + { + if (ch != '.') + result.push_back(ch); + else + /// 0x1F is a unit separator + /// https://github.com/apache/iceberg/blob/70d87f1750627b14b3b25a0216a97db86a786992/open-api/rest-catalog-open-api.yaml#L264 + result.push_back(static_cast(0x1F)); + } + return result; +} + +} + +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 Options & options, DB::ContextPtr context_) + : ICatalog(warehouse_) + , DB::WithContext(context_) + , base_url(correctAPIURI(base_url_)) + , log(&Poco::Logger::get("RestCatalog(" + warehouse_ + ")")) + , auth_scope(options.auth_scope) + , oauth_server_uri(options.oauth_server_uri) + , pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, 1) +{ + if (!options.catalog_credential.empty()) + { + std::tie(client_id, client_secret) = parseCatalogCredential(options.catalog_credential); + update_token_if_expired = true; + } + else if (!options.auth_header.empty()) + auth_header = parseAuthHeader(options.auth_header); + + if (options.enable_sigv4) + { + aws_credentials = pickCredentialsProvider(options.signing_region); + aws_auth_signer + = std::make_shared(aws_credentials, options.signing_name.data(), options.signing_region); + } + + config = loadConfig(); +} + +RestCatalog::Config RestCatalog::loadConfig() +{ + Poco::URI::QueryParameters params = {{"warehouse", warehouse}}; + auto buf = createReadBuffer(CONFIG_ENDPOINT, Poco::Net::HTTPRequest::HTTP_GET, params); + + std::string json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + + LOG_TEST(log, "Received catalog configuration settings: {}", 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(); + parseCatalogConfigurationSettings(defaults_object, result); + + auto overrides_object = object->get("overrides").extract(); + parseCatalogConfigurationSettings(overrides_object, result); + + LOG_TEST(log, "Parsed catalog configuration settings: {}", result.toString()); + return result; +} + +void RestCatalog::parseCatalogConfigurationSettings(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(); +} + +DB::HTTPHeaderEntries RestCatalog::getAuthHeaders( + const std::string & uri, const std::string & method, const std::shared_ptr & payload, bool update_token) const +{ + /// Option 1: user specified auth header manually. + /// Header has format: 'Authorization: '. + if (auth_header.has_value()) + { + return DB::HTTPHeaderEntries{auth_header.value()}; + } + + /// Option 2: user provided grant_type, client_id and client_secret. + /// We would make OAuthClientCredentialsRequest + /// https://github.com/apache/iceberg/blob/3badfe0c1fcf0c0adfc7aa4a10f0b50365c48cf9/open-api/rest-catalog-open-api.yaml#L3498C5-L3498C34 + if (!client_id.empty()) + { + if (!access_token.has_value() || update_token) + { + access_token = retrieveAccessToken(); + } + + DB::HTTPHeaderEntries headers; + headers.emplace_back("Authorization", "Bearer " + access_token.value()); + return headers; + } + + /// Option 3: AWS Sign V4 + if (aws_auth_signer) + { + Aws::Http::URI url{uri}; + Aws::Http::HttpMethod aws_http_method; + if (method == Poco::Net::HTTPRequest::HTTP_POST) + aws_http_method = Aws::Http::HttpMethod::HTTP_POST; + else if (method == Poco::Net::HTTPRequest::HTTP_DELETE) + aws_http_method = Aws::Http::HttpMethod::HTTP_DELETE; + else if (method == Poco::Net::HTTPRequest::HTTP_PUT) + aws_http_method = Aws::Http::HttpMethod::HTTP_PUT; + else if (method == Poco::Net::HTTPRequest::HTTP_HEAD) + aws_http_method = Aws::Http::HttpMethod::HTTP_HEAD; + else if (method == Poco::Net::HTTPRequest::HTTP_PATCH) + aws_http_method = Aws::Http::HttpMethod::HTTP_PATCH; + else + aws_http_method = Aws::Http::HttpMethod::HTTP_GET; + + Aws::Http::Standard::StandardHttpRequest req{url, aws_http_method}; + req.AddContentBody(payload); + if (!aws_auth_signer->SignRequest(req)) + { + throw DB::Exception(DB::ErrorCodes::AWS_ERROR, "Failed to sign REST catalog request"); + } + + DB::HTTPHeaderEntries headers; + for (const auto & header : Aws::Auth::AWSAuthHelper::CanonicalizeHeaders(req.GetHeaders())) + headers.emplace_back(header.first, header.second); + return headers; + } + + return {}; +} + +std::string RestCatalog::retrieveAccessToken() const +{ +#if 0 + static constexpr auto oauth_tokens_endpoint = "oauth/tokens"; + + /// TODO: + /// 1. support oauth2-server-uri + /// https://github.com/apache/iceberg/blob/918f81f3c3f498f46afcea17c1ac9cdc6913cb5c/open-api/rest-catalog-open-api.yaml#L183C82-L183C99 + + 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; + DB::ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback; + if (oauth_server_uri.empty()) + { + url = Poco::URI(base_url / oauth_tokens_endpoint); + + Poco::URI::QueryParameters params = { + {"grant_type", "client_credentials"}, + {"scope", auth_scope}, + {"client_id", client_id}, + {"client_secret", client_secret}, + }; + url.setQueryParameters(params); + } + else + { + url = Poco::URI(oauth_server_uri); + out_stream_callback = [&](std::ostream & os) + { + os << fmt::format( + "grant_type=client_credentials&scope={}&client_id={}&client_secret={}", + auth_scope, client_id, client_secret); + }; + } + + const auto & context = getContext(); + auto wb = DB::BuilderRWBufferFromHTTP(url) + .withConnectionGroup(DB::HTTPConnectionGroupType::HTTP) + .withMethod(Poco::Net::HTTPRequest::HTTP_POST) + .withSettings(context->getReadSettings()) + .withTimeouts(DB::ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), context->getServerSettings())) + .withHostFilter(&context->getRemoteHostFilter()) + .withOutCallback(std::move(out_stream_callback)) + .withSkipNotFound(false) + .withHeaders(headers) + .create(credentials); + + std::string json_str; + readJSONObjectPossiblyInvalid(json_str, *wb); + + 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(); +#else + return ""; +#endif +} + +std::optional RestCatalog::getStorageType() const +{ + if (config.default_base_location.empty()) + return std::nullopt; + return parseStorageTypeFromLocation(config.default_base_location); +} + +std::unique_ptr RestCatalog::createReadBuffer( + const std::string & endpoint, + const std::string & method, + const Poco::URI::QueryParameters & params, + const DB::HTTPHeaderEntries & headers) const +{ + const auto & context = getContext(); + + Poco::URI url(base_url / endpoint); + if (!params.empty()) + url.setQueryParameters(params); + + auto create_buffer = [&](bool update_token) { + auto result_headers = getAuthHeaders(url.toString(), method, nullptr, update_token); + std::move(headers.begin(), headers.end(), std::back_inserter(result_headers)); + + return std::make_unique( + url, + method, + /*out_stream_callback_=*/nullptr, + DB::ConnectionTimeouts::getHTTPTimeouts(context), + credentials, + /*max_redirects=*/0, + DBMS_DEFAULT_BUFFER_SIZE, + getContext()->getReadSettings(), + result_headers, + &getContext()->getRemoteHostFilter(), + /*delay_initialization_=*/false); + }; + + LOG_TEST(log, "Requesting: {}", url.toString()); + + try + { + return create_buffer(false); + } + catch (const DB::HTTPException & e) + { + const auto status = e.getHTTPStatus(); + if (update_token_if_expired + && (status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_UNAUTHORIZED + || status == Poco::Net::HTTPResponse::HTTPStatus::HTTP_FORBIDDEN)) + { + return create_buffer(true); + } + throw; + } +} + +bool RestCatalog::empty(const std::string & namespace_name) const +{ + const auto tables = getTablesImpl(namespace_name, /*limit=*/1); + return tables.empty(); +} + +bool RestCatalog::existsNamespace(const std::string & name) const +{ + const std::string endpoint = base_url / config.prefix / NAMESPACES_ENDPOINT / encodeNamespaceName(name); + + auto auth_headers = getAuthHeaders(endpoint, Poco::Net::HTTPRequest::HTTP_HEAD); + std::vector> headers; + headers.reserve(auth_headers.size()); + for (const auto & header : auth_headers) + headers.emplace_back(header.name, header.value); + + auto resp = DB::sendRequest( + Poco::URI(endpoint), + Poco::Net::HTTPRequest::HTTP_HEAD, + /*query_id=*/"", + /*user=*/"", + /*password=*/"", + /*payload=*/"", + headers, + {/*connection_timeout_=*/{10, 0}, /*send_timeout_=*/{10, 0}, /*receive_timeout_=*/{10, 0}}, + &Poco::Logger::get("IcebergRestCatalog::existsNamespace")); + + if (resp.second < 300) + return true; + + if (resp.second == Poco::Net::HTTPResponse::HTTP_NOT_FOUND) + return false; + + throw DB::Exception( + DB::ErrorCodes::ICEBERG_CATALOG_ERROR, + "Failed to check namespace existence, HTTP response code={}, body={}", + resp.second, + resp.first); +} + +void RestCatalog::createNamespace(const std::string & name [[maybe_unused]]) const +{ +} + +DB::Names RestCatalog::getTables(const std::string & namespace_name) const +{ + return getTablesImpl(namespace_name); +} + +Poco::URI::QueryParameters RestCatalog::createParentNamespaceParams(const std::string & base_namespace) const +{ + return {{"parent", encodeNamespaceName(base_namespace)}}; +} + +DB::Names RestCatalog::getTablesImpl(const std::string & base_namespace, size_t limit) const +{ + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / base_namespace / "tables"; + auto buf = createReadBuffer( + config.prefix / endpoint, + Poco::Net::HTTPRequest::HTTP_GET, + limit == 0 ? Poco::URI::QueryParameters{} : Poco::URI::QueryParameters{{"pageSize", fmt::format("{}", limit)}}); + /// A catalog might not support the pageSize parameter, so we still tell the parseTables function to only parse the limit number of tables. + return parseTables(*buf, limit); +} + +void RestCatalog::createTable( + const std::string & namespace_name, + const std::string & name, + std::optional location, + const DB::NamesAndTypesList & schema, + std::optional partition_spec, + std::optional write_order, + bool stage_create, + std::unordered_map properties) +{ + Poco::JSON::Object payload; + payload.set("name", name); + payload.set("schema", generateIcebergSchema(schema)); + payload.set("stage-create", stage_create); + if (location) + payload.set("location", location.value()); + if (partition_spec) /// partition_spec should not be a string + payload.set("partition-spec", "FIXME"); + if (write_order) /// write_order should not be a string + payload.set("write-order", "FIXME"); + if (!properties.empty()) + { + Poco::JSON::Object props; + for (const auto & kv : properties) + props.set(kv.first, kv.second); + payload.set("properties", props); + } + + auto oss = std::make_shared(); + payload.stringify(*oss); + + const std::string endpoint = base_url / config.prefix / NAMESPACES_ENDPOINT / encodeNamespaceName(namespace_name) / "tables"; + + auto auth_headers = getAuthHeaders(endpoint, Poco::Net::HTTPRequest::HTTP_POST, oss); + std::vector> headers; + headers.reserve(auth_headers.size()); + for (const auto & header : auth_headers) + headers.emplace_back(header.name, header.value); + + auto resp = DB::sendRequest( + Poco::URI(endpoint), + Poco::Net::HTTPRequest::HTTP_POST, + /*query_id=*/"", + /*user=*/"", + /*password=*/"", + oss->str(), + headers, + {/*connection_timeout_=*/{10, 0}, /*send_timeout_=*/{10, 0}, /*receive_timeout_=*/{10, 0}}, + &Poco::Logger::get("IcebergRestCatalog::createTable")); + + if (resp.second >= 300) + throw DB::Exception( + DB::ErrorCodes::ICEBERG_CATALOG_ERROR, "Failed to create table, HTTP response code={}, body={}", resp.second, resp.first); +} + +DB::Names RestCatalog::parseTables(DB::ReadBuffer & buf, size_t limit) const +{ + if (buf.eof()) + return {}; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, buf); + + try + { + Poco::JSON::Parser parser; + Poco::Dynamic::Var json = parser.parse(json_str); + const Poco::JSON::Object::Ptr & object = json.extract(); + + auto identifiers_object = object->get("identifiers").extract(); + if (!identifiers_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse result"); + + DB::Names tables; + tables.reserve(identifiers_object->size()); + for (size_t i = 0; i < identifiers_object->size(); ++i) + { + const auto current_table_json = identifiers_object->get(static_cast(i)).extract(); + const auto table_name = current_table_json->get("name").extract(); + + tables.push_back(table_name); + if (limit != 0 && tables.size() >= limit) + break; + } + + return tables; + } + catch (DB::Exception & e) + { + e.addMessage("while parsing JSON: " + json_str); + throw; + } +} + +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); +} + +void RestCatalog::deleteTable(const std::string & namespace_name, const std::string & table_name) +{ + const std::string endpoint + = base_url / config.prefix / NAMESPACES_ENDPOINT / encodeNamespaceName(namespace_name) / "tables" / table_name; + + auto auth_headers = getAuthHeaders(endpoint, Poco::Net::HTTPRequest::HTTP_DELETE); + std::vector> headers; + headers.reserve(auth_headers.size()); + for (const auto & header : auth_headers) + headers.emplace_back(header.name, header.value); + + auto resp = DB::sendRequest( + Poco::URI(endpoint), + Poco::Net::HTTPRequest::HTTP_DELETE, + /*query_id=*/"", + /*user=*/"", + /*password=*/"", + /*payload=*/"", + headers, + {/*connection_timeout_=*/{10, 0}, /*send_timeout_=*/{10, 0}, /*receive_timeout_=*/{10, 0}}, + &Poco::Logger::get("IcebergRestCatalog::deleteTable")); + + if (resp.second >= 300) + throw DB::Exception( + DB::ErrorCodes::ICEBERG_CATALOG_ERROR, "Failed to delete table, HTTP response code={}, body={}", resp.second, resp.first); +} + +namespace +{ +void extractTableMetadata(const std::string & json_str, TableMetadata & result, Poco::Logger * log) +{ + 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"); + + int format_version = metadata_object->getValue("format-version"); + result.setFormatVersion(format_version); + result.setSequenceNumber(metadata_object->getValue("last-sequence-number")); + result.setTableUUID(metadata_object->getValue("table-uuid")); + + auto snapshot_id = metadata_object->getValue("current-snapshot-id"); + result.setSnapshotID(snapshot_id); + + auto snapshots = metadata_object->get("snapshots"); + if (!snapshots.isEmpty() && snapshots.isArray()) + { + auto snapshots_array = snapshots.extract(); + for (size_t i = 0; i < snapshots_array->size(); i++) + { + auto snapshot_obj = snapshots_array->get(static_cast(i)).extract(); + if (snapshot_obj->getValue("snapshot-id") == snapshot_id) + { + Snapshot snapshot; + snapshot.snapshot_id = snapshot_id; + snapshot.sequence_number = snapshot_obj->getValue("sequence-number"); + snapshot.timestamp_ms = snapshot_obj->getValue("timestamp-ms"); + snapshot.manifest_list = snapshot_obj->getValue("manifest-list"); + + auto summary_obj = snapshot_obj->get("summary").extract(); + snapshot.summary.operation = summary_obj->getValue("operation"); + snapshot.summary.added_files_size = summary_obj->getValue("added-files-size"); + snapshot.summary.added_data_files = summary_obj->getValue("added-data-files"); + snapshot.summary.added_records = summary_obj->getValue("added-records"); + snapshot.summary.total_data_files = summary_obj->getValue("total-data-files"); + snapshot.summary.total_delete_files = summary_obj->getValue("total-delete-files"); + snapshot.summary.total_records = summary_obj->getValue("total-records"); + snapshot.summary.total_files_size = summary_obj->getValue("total-files-size"); + snapshot.summary.total_position_deletes = summary_obj->getValue("total-position-deletes"); + snapshot.summary.total_equality_deletes = summary_obj->getValue("total-equality-deletes"); + + result.setCurrentSnapshot(std::move(snapshot)); + break; + } + } + } + else + { + LOG_INFO(log, "Table metadata has no snapshots."); + } + + std::string location; + if (result.requiresLocation()) + { + location = metadata_object->get("location").extract(); + result.setLocation(location); + } + + if (result.requiresSchema()) + { + auto schema_processor = DB::IcebergSchemaProcessor(); + std::string iceberg_schema_json; + auto id = DB::IcebergMetadata::parseTableSchema(metadata_object, schema_processor, iceberg_schema_json, log); + auto schema = schema_processor.getTimeplusTableSchemaById(id); + result.setSchema(*schema); + result.setSchemaJSON(iceberg_schema_json); + } + + if (result.requiresCredentials() && object->has("config")) + { + auto config_object = object->get("config").extract(); + if (!config_object) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot parse config result"); + + auto storage_type = parseStorageTypeFromLocation(location); + switch (storage_type) + { + case StorageType::S3: + { + static constexpr auto access_key_id_str = "s3.access-key-id"; + static constexpr auto secret_access_key_str = "s3.secret-access-key"; + static constexpr auto session_token_str = "s3.session-token"; + + std::string access_key_id; + std::string secret_access_key; + std::string session_token; + if (config_object->has(access_key_id_str)) + access_key_id = config_object->get(access_key_id_str).extract(); + if (config_object->has(secret_access_key_str)) + secret_access_key = config_object->get(secret_access_key_str).extract(); + if (config_object->has(session_token_str)) + session_token = config_object->get(session_token_str).extract(); + + result.setStorageCredentials(std::make_shared(access_key_id, secret_access_key, session_token)); + break; + } + default: + break; + } + } +} +} + +void RestCatalog::commitTable( + const std::string & namespace_name, + const std::string & name, + const Requirements & requirements, + const Updates & updates, + TableMetadata & table_metadata) +{ + Poco::JSON::Object payload; + + Poco::JSON::Array namespaces; + { + std::vector parts; + splitInto<'.'>(parts, namespace_name); + for (const auto & part : parts) + namespaces.add(part); + } + + Poco::JSON::Object identifier; + identifier.set("namespace", namespaces); + identifier.set("name", name); + payload.set("identifier", identifier); + + Poco::JSON::Array the_requirements; + for (const auto & requirement : requirements) + requirement->apply(the_requirements); + payload.set("requirements", the_requirements); + + Poco::JSON::Array the_updates; + for (const auto & update : updates) + update->apply(the_updates); + payload.set("updates", the_updates); + + auto payload_json = std::make_shared(); + payload.stringify(*payload_json); + + const std::string endpoint = base_url / config.prefix / NAMESPACES_ENDPOINT / encodeNamespaceName(namespace_name) / "tables" / name; + + auto auth_headers = getAuthHeaders(endpoint, Poco::Net::HTTPRequest::HTTP_POST, payload_json); + std::vector> headers; + headers.reserve(auth_headers.size()); + for (const auto & header : auth_headers) + headers.emplace_back(header.name, header.value); + + auto request_body = payload_json->str(); + + auto resp = DB::sendRequest( + Poco::URI(endpoint), + Poco::Net::HTTPRequest::HTTP_POST, + /*query_id=*/"", + /*user=*/"", + /*password=*/"", + request_body, + headers, + {/*connection_timeout_=*/{10, 0}, /*send_timeout_=*/{10, 0}, /*receive_timeout_=*/{10, 0}}, + &Poco::Logger::get("IcebergRestCatalog::commitTable")); + + if (resp.second >= 300) + throw DB::Exception( + DB::ErrorCodes::ICEBERG_CATALOG_ERROR, "Failed to commit table, HTTP response code={}, body={}", resp.second, resp.first); + + extractTableMetadata(resp.first, table_metadata, log); +} + +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->name().data()); + 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); + + DB::HTTPHeaderEntries headers; + if (result.requiresCredentials()) + { + /// Header `X-Iceberg-Access-Delegation` tells catalog to include storage credentials in LoadTableResponse. + /// Value can be one of the two: + /// 1. `vended-credentials` + /// 2. `remote-signing` + /// Currently we support only the first. + /// https://github.com/apache/iceberg/blob/3badfe0c1fcf0c0adfc7aa4a10f0b50365c48cf9/open-api/rest-catalog-open-api.yaml#L1832 + headers.emplace_back("X-Iceberg-Access-Delegation", "vended-credentials"); + } + + const std::string endpoint = std::filesystem::path(NAMESPACES_ENDPOINT) / encodeNamespaceName(namespace_name) / "tables" / table_name; + auto buf = createReadBuffer(config.prefix / endpoint, Poco::Net::HTTPRequest::HTTP_GET, /* params */ {}, headers); + + if (buf->eof()) + { + LOG_TEST(log, "Table doesn't exist (endpoint: {})", endpoint); + return false; + } + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); + +#ifdef DEBUG_OR_SANITIZER_BUILD + /// This log message might contain credentials, + /// so log it only for debugging. + LOG_TEST(log, "Received metadata for table {}: {}", table_name, json_str); +#endif + + extractTableMetadata(json_str, result, log); + + return true; +} + +/* +{ + "identifier": { + "namespace": [ + "gimitest" + ], + "name": "city_locations" + }, + "requirements": [ + { + "type": "assert-ref-snapshot-id", + "ref": "main" + }, + { + "type": "assert-table-uuid", + "uuid": "d13a7cd8-8518-4003-aa1a-4dfc4308119e" + } + ], + "updates": [ + { + "action": "add-snapshot", + "snapshot": { + "snapshot-id": 4488436330293905232, + ["parent-snapshot-id": xxx,] + "sequence-number": 1, + "timestamp-ms": 1738547253392, + "manifest-list": "s3://tp-gimi-test/city_locations/metadata/snap-4488436330293905232-0-e0c4d753-e51b-476a-9415-a61bcd479b83.avro", + "summary": { + "operation": "append", + "added-files-size": "1373", + "added-data-files": "1", + "added-records": "4", + "total-data-files": "1", + "total-delete-files": "0", + "total-records": "4", + "total-files-size": "1373", + "total-position-deletes": "0", + "total-equality-deletes": "0" + }, + "schema-id": 0 + } + }, + { + "action": "set-snapshot-ref", + "ref-name": "main", + "type": "branch", + "snapshot-id": 4488436330293905232 + } + ] +} +*/ + +} + +#endif diff --git a/src/Storages/Iceberg/RestCatalog.h b/src/Storages/Iceberg/RestCatalog.h new file mode 100644 index 00000000000..932cd7c44df --- /dev/null +++ b/src/Storages/Iceberg/RestCatalog.h @@ -0,0 +1,148 @@ +#pragma once + +#include "config.h" + +#if USE_AVRO +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +class ReadBuffer; +} + +namespace Apache::Iceberg +{ + +class RestCatalog final : public ICatalog, private DB::WithContext +{ +public: + struct Options + { + std::string catalog_credential; + std::string auth_scope; + std::string auth_header; + std::string oauth_server_uri; + + bool enable_sigv4{false}; + std::string signing_region; + std::string signing_name; + }; + + explicit RestCatalog(const std::string & warehouse_, const std::string & base_url_, const Options & options, DB::ContextPtr context_); + + ~RestCatalog() override = default; + + bool existsNamespace(const std::string & name) const override; + + void createNamespace(const std::string & name) const override; + + bool empty(const std::string & namespace_name) const override; + + DB::Names getTables(const std::string & namespace_name) const override; + + bool existsTable(const std::string & namespace_name, const std::string & table_name) const override; + + void createTable( + const std::string & namespace_name, + const std::string & name, + std::optional location, + const DB::NamesAndTypesList & schema, + std::optional partition_spec, + std::optional write_order, + bool stage_create, + std::unordered_map properties) override; + + void deleteTable(const std::string & namespace_name, const std::string & name) override; + + void commitTable( + const std::string & namespace_name, + const std::string & name, + const Requirements & requirements, + const Updates & updates, + TableMetadata & table_metadata) 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; + + std::optional getStorageType() const override; + +private: + struct Config + { + /// 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 + /// (in filesystem or object storage). + std::string default_base_location; + + std::string toString() const; + }; + + const std::filesystem::path base_url; + Poco::Logger * log; + + /// Catalog configuration settings from /v1/config endpoint. + Config config; + + /// Auth headers of format: "Authorization": " " + std::optional auth_header; + + /// Parameters for OAuth. + bool update_token_if_expired = false; + std::string client_id; + std::string client_secret; + std::string auth_scope; + std::string oauth_server_uri; + mutable std::optional access_token; + + Poco::Net::HTTPBasicCredentials credentials; + + std::shared_ptr aws_credentials; + std::shared_ptr aws_auth_signer; + + std::unique_ptr createReadBuffer( + const std::string & endpoint, + const std::string & method = Poco::Net::HTTPRequest::HTTP_GET, + const Poco::URI::QueryParameters & params = {}, + const DB::HTTPHeaderEntries & headers = {}) const; + + Poco::URI::QueryParameters createParentNamespaceParams(const std::string & base_namespace) const; + + using StopCondition = std::function; + using ExecuteFunc = std::function; + + void getNamespacesRecursive_deleted( + const std::string & base_namespace, Namespaces & result, StopCondition stop_condition, ExecuteFunc func) const; + + DB::Names getTablesImpl(const std::string & base_namespace, size_t limit = 0) const; + + DB::Names parseTables(DB::ReadBuffer & buf, size_t limit) const; + + bool getTableMetadataImpl(const std::string & namespace_name, const std::string & table_name, TableMetadata & result) const; + + Config loadConfig(); + std::string retrieveAccessToken() const; + DB::HTTPHeaderEntries getAuthHeaders( + const std::string & uri, + const std::string & method = Poco::Net::HTTPRequest::HTTP_GET, + const std::shared_ptr & payload = nullptr, + bool update_token = false) const; + static void parseCatalogConfigurationSettings(const Poco::JSON::Object::Ptr & object, Config & result); + + ThreadPool pool; +}; + +} + +#endif diff --git a/src/Storages/Iceberg/Schema.cpp b/src/Storages/Iceberg/Schema.cpp new file mode 100644 index 00000000000..175682c9bdf --- /dev/null +++ b/src/Storages/Iceberg/Schema.cpp @@ -0,0 +1,265 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB::ErrorCodes +{ +extern const int CANNOT_CONVERT_TYPE; +} + +using namespace DB; +using JsonObjectPtr = std::shared_ptr; +using JsonArrayPtr = std::shared_ptr; + +namespace +{ + +void setFieldType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name = "type"); + +void setNullableType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name) +{ + const auto nested_type = typeid_cast(proton_type.get())->getNestedType(); + setFieldType(field, nested_type, next_field_id, name); +} + +void setArrayType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name) +{ + const auto element_type = typeid_cast(proton_type.get())->getNestedType(); + + Poco::JSON::Object list_json; + list_json.set("type", "list"); + list_json.set("element-id", next_field_id++); + list_json.set("element-required", !element_type->isNullable()); + setFieldType(list_json, element_type, next_field_id, "element"); + + field.set(name, list_json); +} + +void addField(Poco::JSON::Array & fields, const String & name, const DataTypePtr & data_type, int & next_field_id) +{ + Poco::JSON::Object field; + field.set("id", next_field_id++); + field.set("name", name); + + + if (data_type->isNullable()) + { + field.set("required", false); + /// TODO + /// Version 3: set "initial-default" and "write-default" + setFieldType(field, typeid_cast(data_type.get())->getNestedType(), next_field_id); + } + else + { + field.set("required", true); + setFieldType(field, data_type, next_field_id); + } + + fields.add(field); +} + +Poco::JSON::Object newStructType(const DB::NamesAndTypesList & columns, int & next_field_id) +{ + Poco::JSON::Array fields_array; + for (const auto & [name, data_type] : columns) + addField(fields_array, name, data_type, next_field_id); + + Poco::JSON::Object struct_json; + struct_json.set("type", "struct"); + struct_json.set("fields", fields_array); + return struct_json; +} + +void setTupleType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name = "type") +{ + const auto * tuple_type = typeid_cast(proton_type.get()); + const auto & elements = tuple_type->getElements(); + const auto & names = tuple_type->getElementNames(); + + NamesAndTypesList schema; + for (size_t i = 0; const auto & element : elements) + schema.emplace_back(names[i++], element); + + field.set(name, newStructType(schema, next_field_id)); +} + +void setMapType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name) +{ + const auto * map_type = typeid_cast(proton_type.get()); + const auto key_type = map_type->getKeyType(); + const auto value_type = map_type->getValueType(); + + Poco::JSON::Object map_json; + map_json.set("type", "map"); + map_json.set("key-id", next_field_id++); + setFieldType(map_json, key_type, next_field_id, "key"); + map_json.set("value-id", next_field_id++); + map_json.set("value-required", !value_type->isNullable()); + setFieldType(map_json, value_type, next_field_id, "value"); + + field.set(name, map_json); +} + +void setFieldType(Poco::JSON::Object & field, const DataTypePtr & proton_type, int & next_field_id, const std::string & name) +{ + /// Since we also generate timeplusd stream schemas based on Iceberg table schemas, + /// we need to make sure that timeplusd types and Iceberg types are 1:1 mapped. + /// We could consider using structs for timeplusd types which are not supported by Iceberg table format. + /// For example, we can map `ipv4` to `struct { _tp_ipv4: string }`. + switch (proton_type->getTypeId()) + { + case TypeIndex::Bool: + field.set(name, "boolean"); + break; + + /// Integer types + case TypeIndex::UInt8: + [[fallthrough]]; + case TypeIndex::UInt16: + [[fallthrough]]; + case TypeIndex::UInt32: + [[fallthrough]]; + case TypeIndex::UInt64: + [[fallthrough]]; + case TypeIndex::UInt128: + [[fallthrough]]; + case TypeIndex::UInt256: + throw DB::Exception(DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Iceberg table format does not support unsigned integer types"); + + case TypeIndex::Int8: + [[fallthrough]]; + case TypeIndex::Int16: + throw DB::Exception( + DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Iceberg table format does not support signed integer types less than 32-bit"); + case TypeIndex::Int32: + field.set(name, "int"); + break; + case TypeIndex::Int64: + field.set(name, "long"); + break; + case TypeIndex::Int128: + [[fallthrough]]; + case TypeIndex::Int256: + throw DB::Exception( + DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Iceberg table format does not support signed integer types bigger than 64-bit"); + + /// Floating point + case TypeIndex::Float32: + field.set(name, "float"); + break; + case TypeIndex::Float64: + field.set(name, "double"); + break; + + /// String types + case TypeIndex::String: + field.set(name, "string"); + break; + case TypeIndex::FixedString: + { + auto n = typeid_cast(proton_type.get())->getN(); + field.set(name, fmt::format("fixed[{}]", n)); + break; + } + /// Date/time types + /// TODO Version 3: timestamp_ns, timestamptz_ns + case TypeIndex::Date: + field.set(name, "date"); + break; + case TypeIndex::Date32: + /// ClickHouse maps Iceberg date to DataTypeDate, so we keep it the same way for now. + throw DB::Exception(DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Please use date instead of date32"); + case TypeIndex::DateTime: + throw DB::Exception(DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Please use datetime64 instead of datetime"); + case TypeIndex::DateTime64: + { + auto p = typeid_cast(proton_type.get())->getScale(); + if (p > 6) + throw DB::Exception( + DB::ErrorCodes::CANNOT_CONVERT_TYPE, + "{} with scale greater than 6 is not supported, used scale: {}", + proton_type->getName(), + p); + field.set(name, "timestamptz"); + break; + } + // Special types + case TypeIndex::UUID: + field.set(name, "uuid"); + break; + + // Decimal types (require special handling in conversion) + case TypeIndex::Decimal32: + [[fallthrough]]; + case TypeIndex::Decimal64: + [[fallthrough]]; + case TypeIndex::Decimal128: + [[fallthrough]]; + case TypeIndex::Decimal256: + field.set(name, fmt::format("decimal({}, {})", getDecimalPrecision(*proton_type), getDecimalScale(*proton_type))); + break; + + // Complex types (should be handled before this) + case TypeIndex::Array: + setArrayType(field, proton_type, next_field_id, name); + break; + case TypeIndex::Tuple: + setTupleType(field, proton_type, next_field_id, name); + break; + case TypeIndex::Map: + setMapType(field, proton_type, next_field_id, name); + break; + case TypeIndex::Nullable: + setNullableType(field, proton_type, next_field_id, name); + break; + case TypeIndex::LowCardinality: + throw std::runtime_error("Complex type should have been handled earlier: " + proton_type->getName()); + + // Unsupported types + case TypeIndex::Nothing: /// TODO Version 3 + [[fallthrough]]; + case TypeIndex::Enum8: + [[fallthrough]]; + case TypeIndex::Enum16: + [[fallthrough]]; + case TypeIndex::IPv4: + [[fallthrough]]; + case TypeIndex::IPv6: + [[fallthrough]]; + case TypeIndex::Object: + [[fallthrough]]; + case TypeIndex::AggregateFunction: + [[fallthrough]]; + case TypeIndex::Function: + [[fallthrough]]; + case TypeIndex::Interval: + [[fallthrough]]; + case TypeIndex::Set: + throw DB::Exception( + DB::ErrorCodes::CANNOT_CONVERT_TYPE, "Iceberg table format does not support type: {}", proton_type->getName()); + } +} + +} // namespace + +namespace Apache::Iceberg +{ + +Poco::JSON::Object generateIcebergSchema(const NamesAndTypesList & proton_schema) +{ + int next_field_id = 1; + auto schema_json = newStructType(proton_schema, next_field_id); + return schema_json; +} + +} diff --git a/src/Storages/Iceberg/Schema.h b/src/Storages/Iceberg/Schema.h new file mode 100644 index 00000000000..709fb3c6a77 --- /dev/null +++ b/src/Storages/Iceberg/Schema.h @@ -0,0 +1,16 @@ +#pragma once + +namespace DB +{ +class NamesAndTypesList; +} + +namespace Poco::JSON +{ +class Object; +} + +namespace Apache::Iceberg +{ +Poco::JSON::Object generateIcebergSchema(const DB::NamesAndTypesList & proton_schema); +} diff --git a/src/Storages/Iceberg/SchemaProcessor.cpp b/src/Storages/Iceberg/SchemaProcessor.cpp new file mode 100644 index 00000000000..015e8c89fb8 --- /dev/null +++ b/src/Storages/Iceberg/SchemaProcessor.cpp @@ -0,0 +1,428 @@ +#include + +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +extern const int UNSUPPORTED_METHOD; +} + +namespace +{ + +template +bool equals(const T & first, const T & second) +{ + std::stringstream first_string_stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + std::stringstream second_string_stream; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + first.stringify(first_string_stream); + if (!first_string_stream) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "JSON Parsing failed"); + } + second.stringify(second_string_stream); + if (!second_string_stream) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "JSON Parsing failed"); + } + return first_string_stream.str() == second_string_stream.str(); +} + + +bool operator==(const Poco::JSON::Object & first, const Poco::JSON::Object & second) +{ + return equals(first, second); +} + +bool operator==(const Poco::JSON::Array & first, const Poco::JSON::Array & second) +{ + return equals(first, second); +} + +bool schemasAreIdentical(const Poco::JSON::Object & first, const Poco::JSON::Object & second) +{ + static String fields_key = "fields"; + if (!first.isArray(fields_key) || !second.isArray(fields_key)) + return false; + return *(first.getArray(fields_key)) == *(second.getArray(fields_key)); +} + +std::pair parseDecimal(const String & type_name) +{ + DB::ReadBufferFromString buf(std::string_view(type_name.begin() + 8, type_name.end() - 1)); + size_t precision; + size_t scale; + readIntText(precision, buf); + skipWhitespaceIfAny(buf); + assertChar(',', buf); + skipWhitespaceIfAny(buf); + tryReadIntText(scale, buf); + return {precision, scale}; +} + +} + +std::string IcebergSchemaProcessor::default_link{}; + +void IcebergSchemaProcessor::addIcebergTableSchema(Poco::JSON::Object::Ptr schema_ptr) +{ + Int32 schema_id = schema_ptr->getValue("schema-id"); + current_schema_id = schema_id; + if (iceberg_table_schemas_by_ids.contains(schema_id)) + { + chassert(timeplus_table_schemas_by_ids.contains(schema_id)); + chassert(schemasAreIdentical(*iceberg_table_schemas_by_ids.at(schema_id), *schema_ptr)); + } + else + { + iceberg_table_schemas_by_ids[schema_id] = schema_ptr; + auto fields = schema_ptr->get("fields").extract(); + auto timeplus_schema = std::make_shared(); + String current_full_name{}; + for (size_t i = 0; i != fields->size(); ++i) + { + auto field = fields->getObject(static_cast(i)); + auto name = field->getValue("name"); + bool required = field->getValue("required"); + current_full_name = name; + auto type = getFieldType(field, "type", required, current_full_name, true); + timeplus_schema->push_back(NameAndTypePair{name, type}); + timeplus_types_by_source_ids[{schema_id, field->getValue("id")}] = NameAndTypePair{current_full_name, type}; + } + timeplus_table_schemas_by_ids[schema_id] = timeplus_schema; + } + current_schema_id = std::nullopt; +} + +NameAndTypePair IcebergSchemaProcessor::getFieldCharacteristics(Int32 schema_version, Int32 source_id) const +{ + auto it = timeplus_types_by_source_ids.find({schema_version, source_id}); + if (it == timeplus_types_by_source_ids.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Field with source id {} is unknown", source_id); + return it->second; +} + +std::optional IcebergSchemaProcessor::tryGetFieldCharacteristics(Int32 schema_version, Int32 source_id) const +{ + auto it = timeplus_types_by_source_ids.find({schema_version, source_id}); + if (it == timeplus_types_by_source_ids.end()) + return {}; + return it->second; +} + +NamesAndTypesList IcebergSchemaProcessor::tryGetFieldsCharacteristics(Int32 schema_id, const std::vector & source_ids) const +{ + NamesAndTypesList fields; + for (const auto & source_id : source_ids) + { + auto it = timeplus_types_by_source_ids.find({schema_id, source_id}); + if (it != timeplus_types_by_source_ids.end()) + fields.push_back(it->second); + } + return fields; +} + +DataTypePtr IcebergSchemaProcessor::getSimpleType(const String & type_name) +{ + if (type_name == "boolean") + return DataTypeFactory::instance().get("Bool"); + if (type_name == "int") + return std::make_shared(); + if (type_name == "long") + return std::make_shared(); + if (type_name == "float") + return std::make_shared(); + if (type_name == "double") + return std::make_shared(); + if (type_name == "date") + return std::make_shared(); + if (type_name == "time") + return std::make_shared(); + if (type_name == "timestamp") + return std::make_shared(6); + if (type_name == "timestamptz") + return std::make_shared(6, "UTC"); + if (type_name == "string" || type_name == "binary") + return std::make_shared(); + if (type_name == "uuid") + return std::make_shared(); + + if (type_name.starts_with("fixed[") && type_name.ends_with(']')) + { + ReadBufferFromString buf(std::string_view(type_name.begin() + 6, type_name.end() - 1)); + size_t n; + readIntText(n, buf); + return std::make_shared(n); + } + + if (type_name.starts_with("decimal(") && type_name.ends_with(')')) + { + ReadBufferFromString buf(std::string_view(type_name.begin() + 8, type_name.end() - 1)); + auto [precision, scale] = parseDecimal(type_name); + return createDecimal(precision, scale); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); +} + +DataTypePtr +IcebergSchemaProcessor::getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type, String & current_full_name, bool is_subfield_of_root) +{ + String type_name = type->getValue("type"); + if (type_name == "list") + { + bool element_required = type->getValue("element-required"); + auto element_type = getFieldType(type, "element", element_required); + return std::make_shared(element_type); + } + + if (type_name == "map") + { + auto key_type = getFieldType(type, "key", true); + auto value_required = type->getValue("value-required"); + auto value_type = getFieldType(type, "value", value_required); + return std::make_shared(key_type, value_type); + } + + if (type_name == "struct") + { + DataTypes element_types; + Names element_names; + auto fields = type->get("fields").extract(); + element_types.reserve(fields->size()); + element_names.reserve(fields->size()); + for (size_t i = 0; i != fields->size(); ++i) + { + auto field = fields->getObject(static_cast(i)); + element_names.push_back(field->getValue("name")); + auto required = field->getValue("required"); + if (is_subfield_of_root) + { + (current_full_name += ".").append(element_names.back()); + scope_guard guard([&] { current_full_name.resize(current_full_name.size() - element_names.back().size() - 1); }); + element_types.push_back(getFieldType(field, "type", required, current_full_name, true)); + timeplus_types_by_source_ids[{current_schema_id.value(), field->getValue("id")}] + = NameAndTypePair{current_full_name, element_types.back()}; + } + else + { + element_types.push_back(getFieldType(field, "type", required)); + } + } + + return std::make_shared(element_types, element_names); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown Iceberg type: {}", type_name); +} + +DataTypePtr IcebergSchemaProcessor::getFieldType( + const Poco::JSON::Object::Ptr & field, const String & type_key, bool required, String & current_full_name, bool is_subfield_of_root) +{ + if (field->isObject(type_key)) + return getComplexTypeFromObject(field->getObject(type_key), current_full_name, is_subfield_of_root); + + auto type = field->get(type_key); + if (type.isString()) + { + const String & type_name = type.extract(); + auto data_type = getSimpleType(type_name); + return required ? data_type : makeNullable(data_type); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected 'type' field: {}", type.toString()); +} + +/** +* Iceberg allows only three types of primitive type conversion: +* int -> long +* float -> double +* decimal(P, S) -> decimal(P', S) where P' > P +* This function checks if `old_type` and `new_type` satisfy to one of these conditions. +**/ +bool IcebergSchemaProcessor::allowPrimitiveTypeConversion(const String & old_type, const String & new_type) +{ + bool allowed_type_conversion = (old_type == new_type); + allowed_type_conversion |= (old_type == "int") && (new_type == "long"); + allowed_type_conversion |= (old_type == "float") && (new_type == "double"); + if (old_type.starts_with("decimal(") && old_type.ends_with(')') && new_type.starts_with("decimal(") && new_type.ends_with(")")) + { + auto [old_precision, old_scale] = parseDecimal(old_type); + auto [new_precision, new_scale] = parseDecimal(new_type); + allowed_type_conversion |= (old_precision <= new_precision) && (old_scale == new_scale); + } + return allowed_type_conversion; +} + +// Ids are passed only for error logging purposes +std::shared_ptr IcebergSchemaProcessor::getSchemaTransformationDag( + const Poco::JSON::Object::Ptr & old_schema, const Poco::JSON::Object::Ptr & new_schema, Int32 old_id, Int32 new_id) +{ + if (old_id > 0 || new_id > 0) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "IcebergSchemaProcessor::getSchemaTransformationDag is not ready to be called."); + + std::unordered_map> old_schema_entries; + auto old_schema_fields = old_schema->get("fields").extract(); + std::shared_ptr dag = std::make_shared(); + auto & outputs = dag->getOutputs(); + for (size_t i = 0; i != old_schema_fields->size(); ++i) + { + auto field = old_schema_fields->getObject(static_cast(i)); + size_t id = field->getValue("id"); + auto name = field->getValue("name"); + bool required = field->getValue("required"); + old_schema_entries[id] = {field, &dag->addInput(name, getFieldType(field, "type", required))}; + } + auto new_schema_fields = new_schema->get("fields").extract(); + for (size_t i = 0; i != new_schema_fields->size(); ++i) + { + auto field = new_schema_fields->getObject(static_cast(i)); + size_t id = field->getValue("id"); + auto name = field->getValue("name"); + bool required = field->getValue("required"); + auto type = getFieldType(field, "type", required); + auto old_node_it = old_schema_entries.find(id); + if (old_node_it != old_schema_entries.end()) + { + auto [old_json, old_node] = old_node_it->second; + if (field->isObject("type")) + { + if (*old_json != *field) + { + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Schema evolution is not supported for complex types yet, field id is {}, old schema id is {}, new schema id " + "is {}", + id, + old_id, + new_id); + } + else + { + outputs.push_back(old_node); + } + } + else + { + if (old_json->isObject("type")) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Can't cast primitive type to the complex type, field id is {}, old schema id is {}, new schema id is {}", + id, + old_id, + new_id); + } + String old_type = old_json->getValue("type"); + String new_type = field->getValue("type"); + + const ActionsDAG::Node * node = old_node; + if (old_type == new_type) + { + if (old_json->getValue("name") != name) + { + node = &dag->addAlias(*old_node, name); + } + } + else if (allowPrimitiveTypeConversion(old_type, new_type)) + { + /// FIXME + // node = &dag->addCast(*old_node, getFieldType(field, "type", required), name); + } + outputs.push_back(node); + } + } + else + { + if (field->isObject("type")) + { + throw Exception( + ErrorCodes::UNSUPPORTED_METHOD, + "Adding a default column with id {} and complex type is not supported yet. Old schema id is {}, new schema id is " + "{}", + id, + old_id, + new_id); + } + if (!type->isNullable()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot add a column with id {} with required values to the table during schema evolution. This is forbidden by " + "Iceberg format specification. Old schema id is {}, new " + "schema id is {}", + id, + old_id, + new_id); + } + ColumnPtr default_type_column = type->createColumnConstWithDefaultValue(0); + const auto & constant = dag->addColumn({default_type_column, type, name}); + outputs.push_back(&constant); + } + } + return dag; +} + +std::shared_ptr IcebergSchemaProcessor::getSchemaTransformationDagByIds(Int32 old_id, Int32 new_id) +{ + if (old_id == new_id) + { + return nullptr; + } + std::lock_guard lock(mutex); + auto required_transform_dag_it = transform_dags_by_ids.find({old_id, new_id}); + if (required_transform_dag_it != transform_dags_by_ids.end()) + { + return required_transform_dag_it->second; + } + + auto old_schema_it = iceberg_table_schemas_by_ids.find(old_id); + if (old_schema_it == iceberg_table_schemas_by_ids.end()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Schema with schema-id {} is unknown", old_id); + } + auto new_schema_it = iceberg_table_schemas_by_ids.find(new_id); + if (new_schema_it == iceberg_table_schemas_by_ids.end()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Schema with schema-id {} is unknown", new_id); + } + return transform_dags_by_ids[{old_id, new_id}] + = getSchemaTransformationDag(old_schema_it->second, new_schema_it->second, old_id, new_id); +} + +std::shared_ptr IcebergSchemaProcessor::getTimeplusTableSchemaById(Int32 id) +{ + auto it = timeplus_table_schemas_by_ids.find(id); + if (it == timeplus_table_schemas_by_ids.end()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Schema with id {} is unknown", id); + return it->second; +} + +} diff --git a/src/Storages/Iceberg/SchemaProcessor.h b/src/Storages/Iceberg/SchemaProcessor.h new file mode 100644 index 00000000000..2b5084fc8e3 --- /dev/null +++ b/src/Storages/Iceberg/SchemaProcessor.h @@ -0,0 +1,107 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +/** + * Iceberg supports the following data types (see https://iceberg.apache.org/spec/#schemas-and-data-types): + * - Primitive types: + * - boolean + * - int + * - long + * - float + * - double + * - decimal(P, S) + * - date + * - time (time of day in microseconds since midnight) + * - timestamp (in microseconds since 1970-01-01) + * - timestamptz (timestamp with timezone, stores values in UTC timezone) + * - string + * - uuid + * - fixed(L) (fixed-length byte array of length L) + * - binary + * - Complex types: + * - struct(field1: Type1, field2: Type2, ...) (tuple of typed values) + * - list(nested_type) + * - map(Key, Value) + * + * Example of table schema in metadata: + * { + * "type" : "struct", + * "schema-id" : 0, + * "fields" : [ + * { + * "id" : 1, + * "name" : "id", + * "required" : false, + * "type" : "long" + * }, + * { + * "id" : 2, + * "name" : "array", + * "required" : false, + * "type" : { + * "type" : "list", + * "element-id" : 5, + * "element" : "int", + * "element-required" : false + * }, + * { + * "id" : 3, + * "name" : "data", + * "required" : false, + * "type" : "binary" + * } + * } + */ +class IcebergSchemaProcessor +{ + static std::string default_link; + + using Node = ActionsDAG::Node; + +public: + void addIcebergTableSchema(Poco::JSON::Object::Ptr schema_ptr); + std::shared_ptr getTimeplusTableSchemaById(Int32 id); + std::shared_ptr getSchemaTransformationDagByIds(Int32 old_id, Int32 new_id); + NameAndTypePair getFieldCharacteristics(Int32 schema_version, Int32 source_id) const; + std::optional tryGetFieldCharacteristics(Int32 schema_version, Int32 source_id) const; + NamesAndTypesList tryGetFieldsCharacteristics(Int32 schema_id, const std::vector & source_ids) const; + +private: + std::unordered_map iceberg_table_schemas_by_ids; + std::unordered_map> timeplus_table_schemas_by_ids; + std::map, std::shared_ptr> transform_dags_by_ids; + mutable std::map, NameAndTypePair> timeplus_types_by_source_ids; + + NamesAndTypesList getSchemaType(const Poco::JSON::Object::Ptr & schema); + DataTypePtr getComplexTypeFromObject(const Poco::JSON::Object::Ptr & type, String & current_full_name, bool is_subfield_of_root); + DataTypePtr getFieldType( + const Poco::JSON::Object::Ptr & field, + const String & type_key, + bool required, + String & current_full_name = default_link, + bool is_subfield_of_root = false); + + static DataTypePtr getSimpleType(const String & type_name); + + std::optional current_schema_id = 0; + + bool allowPrimitiveTypeConversion(const String & old_type, const String & new_type); + const Node * getDefaultNodeForField(const Poco::JSON::Object::Ptr & field); + + std::shared_ptr getSchemaTransformationDag( + const Poco::JSON::Object::Ptr & old_schema, const Poco::JSON::Object::Ptr & new_schema, Int32 old_id, Int32 new_id); + + std::mutex mutex; +}; + +} diff --git a/src/Storages/Iceberg/Snapshot.h b/src/Storages/Iceberg/Snapshot.h new file mode 100644 index 00000000000..32ee9ebc7b8 --- /dev/null +++ b/src/Storages/Iceberg/Snapshot.h @@ -0,0 +1,40 @@ +#include "config.h" + +#if USE_AVRO + +#include + +/// Represents a snapshot in the Iceberg table format. +struct Snapshot { + /// A unique long ID for the snapshot. + int64_t snapshot_id; + + /// The snapshot ID of the snapshot's parent. + /// This field is omitted for snapshots with no parent. + int64_t parent_snapshot_id; + + /// A monotonically increasing long that tracks the order of changes to a table. + int64_t sequence_number; + + /// A timestamp in milliseconds when the snapshot was created. + /// Used for garbage collection and table inspection. + int64_t timestamp_ms; + + /// The location of a manifest list for this snapshot + /// that tracks manifest files with additional metadata. + std::string manifest_list; + + /// A map summarizing the snapshot changes. + /// The "operation" field is required and describes the type of operation. + std::unordered_map summary; + + /// ID of the table's current schema when the snapshot was created. + int schema_id; + + /// The first `_row_id` assigned to the first row in the first data file in the first manifest. + int64_t first_row_id; + + /// Sum of the `added_rows_count` from all manifests added in this snapshot. + int64_t added_rows; +}; +#endif diff --git a/src/Storages/Iceberg/StorageCredentials.h b/src/Storages/Iceberg/StorageCredentials.h new file mode 100644 index 00000000000..087f1536d76 --- /dev/null +++ b/src/Storages/Iceberg/StorageCredentials.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace Apache::Iceberg +{ + +class IStorageCredentials +{ +public: + virtual ~IStorageCredentials() = default; + + virtual void addCredentialsToSettings(DB::ASTSetQuery & settings) const = 0; +}; + +class S3Credentials final : public IStorageCredentials +{ +public: + /// TODO: support region as well. + S3Credentials(const std::string & access_key_id_, const std::string & secret_access_key_, const std::string session_token_) + : access_key_id(access_key_id_), secret_access_key(secret_access_key_), session_token(session_token_) + { + } + + void addCredentialsToSettings(DB::ASTSetQuery & settings) const override + { + settings.changes.setSetting("access_key_id", access_key_id); + settings.changes.setSetting("secret_access_key", secret_access_key); + settings.changes.setSetting("session_token", session_token); + } + +private: + std::string access_key_id; + std::string secret_access_key; + std::string session_token; +}; + +} diff --git a/src/Storages/Iceberg/Update.cpp b/src/Storages/Iceberg/Update.cpp new file mode 100644 index 00000000000..a67084cf013 --- /dev/null +++ b/src/Storages/Iceberg/Update.cpp @@ -0,0 +1,47 @@ +#include + +#include + +namespace Apache::Iceberg +{ + +void AddSnopshotUpdate::apply(Poco::JSON::Array & updates) const +{ + Poco::JSON::Object summary_obj; + summary_obj.set("operation", snapshot.summary.operation); + summary_obj.set("added-files-size", snapshot.summary.added_files_size); + summary_obj.set("added-data-files", snapshot.summary.added_data_files); + summary_obj.set("added-records", snapshot.summary.added_records); + summary_obj.set("total-data-files", snapshot.summary.total_data_files); + summary_obj.set("total-delete-files", snapshot.summary.total_delete_files); + summary_obj.set("total-records", snapshot.summary.total_records); + summary_obj.set("total-files-size", snapshot.summary.total_files_size); + summary_obj.set("total-position-deletes", snapshot.summary.total_position_deletes); + summary_obj.set("total-equality-deletes", snapshot.summary.total_equality_deletes); + + Poco::JSON::Object snapshot_obj; + snapshot_obj.set("snapshot-id", snapshot.snapshot_id); + if (snapshot.parent_snapshot_id) + snapshot_obj.set("parent-snapshot-id", *snapshot.parent_snapshot_id); + snapshot_obj.set("sequence-number", snapshot.sequence_number); + snapshot_obj.set("timestamp-ms", snapshot.timestamp_ms); + snapshot_obj.set("manifest-list", snapshot.manifest_list); + snapshot_obj.set("summary", summary_obj); + + Poco::JSON::Object obj; + obj.set("action", action); + obj.set("snapshot", snapshot_obj); + updates.add(obj); +} + +void SetSnapshotRefUpdate::apply(Poco::JSON::Array & updates) const +{ + Poco::JSON::Object obj; + obj.set("action", action); + obj.set("ref-name", snapshot_ref.name); + obj.set("type", snapshot_ref.type); + obj.set("snapshot-id", snapshot_ref.snapshot_id); + updates.add(obj); +} + +} diff --git a/src/Storages/Iceberg/Update.h b/src/Storages/Iceberg/Update.h new file mode 100644 index 00000000000..10e56b0bfc4 --- /dev/null +++ b/src/Storages/Iceberg/Update.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include + +namespace Poco::JSON +{ +class Array; +} + +namespace Apache::Iceberg +{ + +class Update +{ +public: + explicit Update(const std::string & action_) : action(action_) { } + virtual ~Update() = default; + + virtual void apply(Poco::JSON::Array & updates) const = 0; + + const std::string & getAction() const { return action; } + +protected: + const std::string action; +}; + +struct Snapshot +{ + /// Only `operation` is required. + struct Summary + { + std::string operation; + std::string added_files_size; + std::string added_data_files; + std::string added_records; + std::string total_data_files; + std::string total_delete_files; + std::string total_records; + std::string total_files_size; + std::string total_position_deletes; + std::string total_equality_deletes; + /// There are even more fields not listed here + /// TODO at least should add engine-name and engine-version + }; + + int64_t snapshot_id{-1}; + std::optional parent_snapshot_id; + uint64_t sequence_number{0}; + int64_t timestamp_ms{0}; + std::string manifest_list; + Summary summary; + + bool isValid() const { return snapshot_id > 0; } +}; + +class AddSnopshotUpdate : public Update +{ +public: + explicit AddSnopshotUpdate(const Snapshot & snapshot_) : Update("add-snapshot"), snapshot(snapshot_) { } + ~AddSnopshotUpdate() override = default; + + void apply(Poco::JSON::Array & updates) const override; + +private: + Snapshot snapshot; +}; + +struct SnapshotReference +{ + std::string name; + std::string type; + int64_t snapshot_id; +}; + +class SetSnapshotRefUpdate : public Update +{ +public: + explicit SetSnapshotRefUpdate(const SnapshotReference & snapshot_ref_) : Update("set-snapshot-ref"), snapshot_ref(snapshot_ref_) { } + ~SetSnapshotRefUpdate() override = default; + + void apply(Poco::JSON::Array & updates) const override; + +private: + SnapshotReference snapshot_ref; +}; + +using UpdatePtr = std::unique_ptr; +using Updates = std::vector; + +} diff --git a/src/Storages/Iceberg/schemas/manifest.json b/src/Storages/Iceberg/schemas/manifest.json new file mode 100644 index 00000000000..eb28702fae8 --- /dev/null +++ b/src/Storages/Iceberg/schemas/manifest.json @@ -0,0 +1,131 @@ +{ + "type": "record", + "name": "manifest_file", + "fields": [ + { + "name": "manifest_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 500 + }, + { + "name": "manifest_length", + "type": "long", + "field-id": 501 + }, + { + "name": "partition_spec_id", + "type": "int", + "field-id": 502 + }, + { + "name": "content", + "type": "int", + "field-id": 517 + }, + { + "name": "sequence_number", + "type": "long", + "field-id": 515 + }, + { + "name": "min_sequence_number", + "type": "long", + "field-id": 516 + }, + { + "name": "added_snapshot_id", + "type": "long", + "field-id": 503 + }, + { + "name": "added_files_count", + "type": "int", + "field-id": 504 + }, + { + "name": "existing_files_count", + "type": "int", + "field-id": 505 + }, + { + "name": "deleted_files_count", + "type": "int", + "field-id": 506 + }, + { + "name": "added_rows_count", + "type": "long", + "field-id": 512 + }, + { + "name": "existing_rows_count", + "type": "long", + "field-id": 513 + }, + { + "name": "deleted_rows_count", + "type": "long", + "field-id": 514 + }, + { + "name": "partitions", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "r508", + "fields": [ + { + "name": "contains_null", + "type": "boolean", + "field-id": 509 + }, + { + "name": "contains_nan", + "type": [ + "null", + "boolean" + ], + "default": null, + "field-id": 518 + }, + { + "name": "lower_bound", + "type": [ + "null", + "bytes" + ], + "default": null, + "field-id": 510 + }, + { + "name": "upper_bound", + "type": [ + "null", + "bytes" + ], + "default": null, + "field-id": 511 + } + ] + }, + "element-id": 508 + } + ], + "default": null, + "field-id": 507 + }, + { + "name": "key_metadata", + "type": [ + "null", + "bytes" + ], + "default": null, + "field-id": 519 + } + ] +} diff --git a/src/Storages/Iceberg/schemas/manifest_entry.json b/src/Storages/Iceberg/schemas/manifest_entry.json new file mode 100644 index 00000000000..978f3027099 --- /dev/null +++ b/src/Storages/Iceberg/schemas/manifest_entry.json @@ -0,0 +1,310 @@ +{ + "type": "record", + "name": "manifest_entry", + "fields": [ + { + "name": "status", + "type": "int", + "field-id": 0 + }, + { + "name": "snapshot_id", + "type": [ + "null", + "long" + ], + "default": null, + "field-id": 1 + }, + { + "name": "sequence_number", + "type": [ + "null", + "long" + ], + "default": null, + "field-id": 3 + }, + { + "name": "file_sequence_number", + "type": [ + "null", + "long" + ], + "default": null, + "field-id": 4 + }, + { + "name": "data_file", + "type": { + "type": "record", + "name": "r2", + "fields": [ + { + "name": "content", + "type": "int", + "doc": "Contents of the file: 0=data, 1=position deletes, 2=equality deletes", + "field-id": 134 + }, + { + "name": "file_path", + "type": "string", + "doc": "Location URI with FS scheme", + "field-id": 100 + }, + { + "name": "file_format", + "type": "string", + "doc": "File format name: avro, orc, or parquet", + "field-id": 101 + }, + { + "name": "partition", + "type": { + "type": "record", + "name": "r102", + "fields": [] + }, + "doc": "Partition data tuple, schema based on the partition spec", + "field-id": 102 + }, + { + "name": "record_count", + "type": "long", + "doc": "Number of records in the file", + "field-id": 103 + }, + { + "name": "file_size_in_bytes", + "type": "long", + "doc": "Total file size in bytes", + "field-id": 104 + }, + { + "name": "column_sizes", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k117_v118", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 117 + }, + { + "name": "value", + "type": "long", + "field-id": 118 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to total size on disk", + "default": null, + "field-id": 108 + }, + { + "name": "value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k119_v120", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 119 + }, + { + "name": "value", + "type": "long", + "field-id": 120 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to total count, including null and NaN", + "default": null, + "field-id": 109 + }, + { + "name": "null_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k121_v122", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 121 + }, + { + "name": "value", + "type": "long", + "field-id": 122 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to null value count", + "default": null, + "field-id": 110 + }, + { + "name": "nan_value_counts", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k138_v139", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 138 + }, + { + "name": "value", + "type": "long", + "field-id": 139 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to number of NaN values in the column", + "default": null, + "field-id": 137 + }, + { + "name": "lower_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k126_v127", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 126 + }, + { + "name": "value", + "type": "bytes", + "field-id": 127 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to lower bound", + "default": null, + "field-id": 125 + }, + { + "name": "upper_bounds", + "type": [ + "null", + { + "type": "array", + "items": { + "type": "record", + "name": "k129_v130", + "fields": [ + { + "name": "key", + "type": "int", + "field-id": 129 + }, + { + "name": "value", + "type": "bytes", + "field-id": 130 + } + ] + }, + "logicalType": "map" + } + ], + "doc": "Map of column id to upper bound", + "default": null, + "field-id": 128 + }, + { + "name": "key_metadata", + "type": [ + "null", + "bytes" + ], + "doc": "Encryption key metadata blob", + "default": null, + "field-id": 131 + }, + { + "name": "split_offsets", + "type": [ + "null", + { + "type": "array", + "items": "long", + "element-id": 133 + } + ], + "doc": "Splittable offsets", + "default": null, + "field-id": 132 + }, + { + "name": "equality_ids", + "type": [ + "null", + { + "type": "array", + "items": "int", + "element-id": 136 + } + ], + "doc": "Equality comparison field IDs", + "default": null, + "field-id": 135 + }, + { + "name": "sort_order_id", + "type": [ + "null", + "int" + ], + "doc": "Sort order ID", + "default": null, + "field-id": 140 + } + ] + }, + "field-id": 2 + } + ] +} diff --git a/src/Storages/NamedCollections/NamedCollectionConfiguration.cpp b/src/Storages/NamedCollections/NamedCollectionConfiguration.cpp new file mode 100644 index 00000000000..6875458958b --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionConfiguration.cpp @@ -0,0 +1,185 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; +} + +namespace NamedCollectionConfiguration +{ + +template T getConfigValue( + const Poco::Util::AbstractConfiguration & config, + const std::string & path) +{ + return getConfigValueOrDefault(config, path); +} + +template T getConfigValueOrDefault( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T * default_value) +{ + if (!config.has(path)) + { + if (!default_value) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); + return *default_value; + } + + try + { + if constexpr (std::is_same_v) + return config.getString(path); + else if constexpr (std::is_same_v) + return config.getUInt64(path); + else if constexpr (std::is_same_v) + return config.getInt64(path); + else if constexpr (std::is_same_v) + return config.getDouble(path); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unsupported type in getConfigValueOrDefault(). " + "Supported types are String, UInt64, Int64, Float64"); + } + catch (const Poco::SyntaxException &) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot extract {} from {}", + toString(magic_enum::enum_name(Field::TypeToEnum>::value)), + path); + } +} + +template void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T & value, + bool update) +{ + if (!update && config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path); + + if constexpr (std::is_same_v) + config.setString(path, value); + else if constexpr (std::is_same_v) + config.setUInt64(path, value); + else if constexpr (std::is_same_v) + config.setInt64(path, value); + else if constexpr (std::is_same_v) + config.setDouble(path, value); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unsupported type in setConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); +} + +template void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path) +{ + if (!from_config.has(from_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path); + + if (to_config.has(to_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path); + + if constexpr (std::is_same_v) + to_config.setString(to_path, from_config.getString(from_path)); + else if constexpr (std::is_same_v) + to_config.setUInt64(to_path, from_config.getUInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setInt64(to_path, from_config.getInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setDouble(to_path, from_config.getDouble(from_path)); + else + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Unsupported type in copyConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); +} + +void removeConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path) +{ + if (!config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); + config.remove(path); +} + +ConfigurationPtr createEmptyConfiguration(const std::string & root_name) +{ + using DocumentPtr = Poco::AutoPtr; + using ElementPtr = Poco::AutoPtr; + + DocumentPtr xml_document(new Poco::XML::Document()); + ElementPtr root_element(xml_document->createElement(root_name)); + xml_document->appendChild(root_element); + + ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document)); + return config; +} + +ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings) +{ + namespace Configuration = NamedCollectionConfiguration; + + auto config = Configuration::createEmptyConfiguration(root_name); + for (const auto & [name, value] : settings) + Configuration::setConfigValue(*config, name, convertFieldToString(value)); + + return config; +} + +template String getConfigValue(const Poco::Util::AbstractConfiguration & config, + const std::string & path); +template UInt64 getConfigValue(const Poco::Util::AbstractConfiguration & config, + const std::string & path); +template Int64 getConfigValue(const Poco::Util::AbstractConfiguration & config, + const std::string & path); +template Float64 getConfigValue(const Poco::Util::AbstractConfiguration & config, + const std::string & path); + +template String getConfigValueOrDefault(const Poco::Util::AbstractConfiguration & config, + const std::string & path, const String * default_value); +template UInt64 getConfigValueOrDefault(const Poco::Util::AbstractConfiguration & config, + const std::string & path, const UInt64 * default_value); +template Int64 getConfigValueOrDefault(const Poco::Util::AbstractConfiguration & config, + const std::string & path, const Int64 * default_value); +template Float64 getConfigValueOrDefault(const Poco::Util::AbstractConfiguration & config, + const std::string & path, const Float64 * default_value); + +template void setConfigValue(Poco::Util::AbstractConfiguration & config, + const std::string & path, const String & value, bool update); +template void setConfigValue(Poco::Util::AbstractConfiguration & config, + const std::string & path, const UInt64 & value, bool update); +template void setConfigValue(Poco::Util::AbstractConfiguration & config, + const std::string & path, const Int64 & value, bool update); +template void setConfigValue(Poco::Util::AbstractConfiguration & config, + const std::string & path, const Float64 & value, bool update); + +template void copyConfigValue(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, const std::string & to_path); +template void copyConfigValue(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, const std::string & to_path); +template void copyConfigValue(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, const std::string & to_path); +template void copyConfigValue(const Poco::Util::AbstractConfiguration & from_config, const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, const std::string & to_path); +} + +} diff --git a/src/Storages/NamedCollections/NamedCollectionConfiguration.h b/src/Storages/NamedCollections/NamedCollectionConfiguration.h new file mode 100644 index 00000000000..7478dcf2d9a --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionConfiguration.h @@ -0,0 +1,44 @@ +#pragma once +#include + +namespace DB +{ + +using ConfigurationPtr = Poco::AutoPtr; +class SettingsChanges; + +namespace NamedCollectionConfiguration +{ + +ConfigurationPtr createEmptyConfiguration(const std::string & root_name); + +template T getConfigValue( + const Poco::Util::AbstractConfiguration & config, + const std::string & path); + +template T getConfigValueOrDefault( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T * default_value = nullptr); + +template void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T & value, + bool update = false); + +template void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path); + +void removeConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path); + +ConfigurationPtr createConfiguration(const std::string & root_name, const SettingsChanges & settings); + +} + +} diff --git a/src/Storages/NamedCollections/NamedCollectionUtils.cpp b/src/Storages/NamedCollections/NamedCollectionUtils.cpp new file mode 100644 index 00000000000..c4caa5c95f6 --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionUtils.cpp @@ -0,0 +1,434 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace fs = std::filesystem; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int NAMED_COLLECTION_DOESNT_EXIST; + extern const int BAD_ARGUMENTS; +} + +namespace NamedCollectionUtils +{ + +class LoadFromConfig +{ +private: + const Poco::Util::AbstractConfiguration & config; + +public: + explicit LoadFromConfig(const Poco::Util::AbstractConfiguration & config_) + : config(config_) {} + + std::vector listCollections() const + { + Poco::Util::AbstractConfiguration::Keys collections_names; + config.keys(NAMED_COLLECTIONS_CONFIG_PREFIX, collections_names); + return collections_names; + } + + NamedCollectionsMap getAll() const + { + NamedCollectionsMap result; + for (const auto & collection_name : listCollections()) + { + if (result.contains(collection_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Found duplicate named collection `{}`", + collection_name); + } + result.emplace(collection_name, get(collection_name)); + } + return result; + } + + MutableNamedCollectionPtr get(const std::string & collection_name) const + { + const auto collection_prefix = getCollectionPrefix(collection_name); + std::queue enumerate_input; + std::set enumerate_result; + + enumerate_input.push(collection_prefix); + collectKeys(config, std::move(enumerate_input), enumerate_result); + + /// Collection does not have any keys. + /// (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1 + && *enumerate_result.begin() == collection_prefix; + std::set> keys; + if (!collection_is_empty) + { + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + for (const auto & path : enumerate_result) + keys.emplace(path.substr(collection_prefix.size() + 1)); + } + + return NamedCollection::create( + config, collection_name, collection_prefix, keys, SourceId::CONFIG, /* is_mutable */false); + } + +private: + static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; + + static std::string getCollectionPrefix(const std::string & collection_name) + { + return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); + } + + /// Enumerate keys paths of the config recursively. + /// E.g. if `enumerate_paths` = {"root.key1"} and config like + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4" + static void collectKeys( + const Poco::Util::AbstractConfiguration & config, + std::queue enumerate_paths, + std::set & result) + { + if (enumerate_paths.empty()) + return; + + auto initial_paths = std::move(enumerate_paths); + enumerate_paths = {}; + while (!initial_paths.empty()) + { + auto path = initial_paths.front(); + initial_paths.pop(); + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(path, keys); + + if (keys.empty()) + { + result.insert(path); + } + else + { + for (const auto & key : keys) + enumerate_paths.emplace(path + '.' + key); + } + } + + collectKeys(config, enumerate_paths, result); + } +}; + + +class LoadFromSQL : private WithContext +{ +private: + const std::string metadata_path; + +public: + explicit LoadFromSQL(ContextPtr context_) + : WithContext(context_) + , metadata_path( + fs::canonical(context_->getPath()) / NAMED_COLLECTIONS_METADATA_DIRECTORY) + { + if (fs::exists(metadata_path)) + cleanUp(); + else + fs::create_directories(metadata_path); + } + + std::vector listCollections() const + { + std::vector collection_names; + fs::directory_iterator it{metadata_path}; + for (; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".sql") + { + collection_names.push_back(it->path().stem()); + } + else + { + LOG_WARNING( + &Poco::Logger::get("NamedCollectionsLoadFromSQL"), + "Unexpected file {} in named collections directory", + current_path.filename().string()); + } + } + return collection_names; + } + + NamedCollectionsMap getAll() const + { + NamedCollectionsMap result; + for (const auto & collection_name : listCollections()) + { + if (result.contains(collection_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Found duplicate named collection `{}`", + collection_name); + } + result.emplace(collection_name, get(collection_name)); + } + return result; + } + + MutableNamedCollectionPtr get(const std::string & collection_name) const + { + const auto query = readCreateQueryFromMetadata( + getMetadataPath(collection_name), + getContext()->getSettingsRef()); + return createNamedCollectionFromAST(query); + } + + MutableNamedCollectionPtr create(const ASTCreateNamedCollectionQuery & query) + { + writeCreateQueryToMetadata( + query, + getMetadataPath(query.collection_name), + getContext()->getSettingsRef()); + + return createNamedCollectionFromAST(query); + } + + void update(const ASTAlterNamedCollectionQuery & query) + { + const auto path = getMetadataPath(query.collection_name); + auto create_query = readCreateQueryFromMetadata(path, getContext()->getSettings()); + + std::unordered_map result_changes_map; + for (const auto & [name, value] : query.changes) + { + auto [it, inserted] = result_changes_map.emplace(name, value); + if (!inserted) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Value with key `{}` is used twice in the SET query", + name, query.collection_name); + } + } + + for (const auto & [name, value] : create_query.changes) + result_changes_map.emplace(name, value); + + for (const auto & delete_key : query.delete_keys) + { + auto it = result_changes_map.find(delete_key); + if (it == result_changes_map.end()) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot delete key `{}` because it does not exist in collection", + delete_key); + } + else + result_changes_map.erase(it); + } + + create_query.changes.clear(); + for (const auto & [name, value] : result_changes_map) + create_query.changes.emplace_back(name, value); + + writeCreateQueryToMetadata( + create_query, + getMetadataPath(query.collection_name), + getContext()->getSettingsRef(), + true); + } + + void remove(const std::string & collection_name) + { + if (!removeIfExists(collection_name)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "Cannot remove collection `{}`, because it doesn't exist", + collection_name); + } + } + + bool removeIfExists(const std::string & collection_name) + { + auto collection_path = getMetadataPath(collection_name); + if (fs::exists(collection_path)) + { + fs::remove(collection_path); + return true; + } + return false; + } + +private: + static constexpr auto NAMED_COLLECTIONS_METADATA_DIRECTORY = "named_collections"; + + static MutableNamedCollectionPtr createNamedCollectionFromAST( + const ASTCreateNamedCollectionQuery & query) + { + const auto & collection_name = query.collection_name; + const auto config = NamedCollectionConfiguration::createConfiguration( + collection_name, query.changes); + + std::set> keys; + for (const auto & [name, _] : query.changes) + keys.insert(name); + + return NamedCollection::create( + *config, collection_name, "", keys, SourceId::SQL, /* is_mutable */true); + } + + std::string getMetadataPath(const std::string & collection_name) const + { + return fs::path(metadata_path) / (escapeForFileName(collection_name) + ".sql"); + } + + /// Delete .tmp files. They could be left undeleted in case of + /// some exception or abrupt server restart. + void cleanUp() + { + fs::directory_iterator it{metadata_path}; + std::vector files_to_remove; + for (; it != fs::directory_iterator{}; ++it) + { + const auto & current_path = it->path(); + if (current_path.extension() == ".tmp") + files_to_remove.push_back(current_path); + } + for (const auto & file : files_to_remove) + fs::remove(file); + } + + static ASTCreateNamedCollectionQuery readCreateQueryFromMetadata( + const std::string & path, + const Settings & settings) + { + ReadBufferFromFile in(path); + std::string query; + readStringUntilEOF(query, in); + + ParserCreateNamedCollectionQuery parser; + auto ast = parseQuery(parser, query, "in file " + path, 0, settings.max_parser_depth); + const auto & create_query = ast->as(); + return create_query; + } + + static void writeCreateQueryToMetadata( + const ASTCreateNamedCollectionQuery & query, + const std::string & path, + const Settings & settings, + bool replace = false) + { + if (!replace && fs::exists(path)) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "Metadata file {} for named collection already exists", + path); + } + + auto tmp_path = path + ".tmp"; + String formatted_query = serializeAST(query); + WriteBufferFromFile out(tmp_path, formatted_query.size(), O_WRONLY | O_CREAT | O_EXCL); + writeString(formatted_query, out); + + out.next(); + if (settings.fsync_metadata) + out.sync(); + out.close(); + + fs::rename(tmp_path, path); + } +}; + +std::unique_lock lockNamedCollectionsTransaction() +{ + static std::mutex transaction_lock; + return std::unique_lock(transaction_lock); +} + +void loadFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + auto lock = lockNamedCollectionsTransaction(); + NamedCollectionFactory::instance().add(LoadFromConfig(config).getAll()); +} + +void reloadFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + auto lock = lockNamedCollectionsTransaction(); + auto collections = LoadFromConfig(config).getAll(); + auto & instance = NamedCollectionFactory::instance(); + instance.removeById(SourceId::CONFIG); + instance.add(collections); +} + +void loadFromSQL(ContextPtr context) +{ + auto lock = lockNamedCollectionsTransaction(); + NamedCollectionFactory::instance().add(LoadFromSQL(context).getAll()); +} + +void removeFromSQL(const std::string & collection_name, ContextPtr context) +{ + auto lock = lockNamedCollectionsTransaction(); + LoadFromSQL(context).remove(collection_name); + NamedCollectionFactory::instance().remove(collection_name); +} + +void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context) +{ + auto lock = lockNamedCollectionsTransaction(); + LoadFromSQL(context).removeIfExists(collection_name); + NamedCollectionFactory::instance().removeIfExists(collection_name); +} + +void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context) +{ + auto lock = lockNamedCollectionsTransaction(); + NamedCollectionFactory::instance().add(query.collection_name, LoadFromSQL(context).create(query)); +} + +void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context) +{ + auto lock = lockNamedCollectionsTransaction(); + LoadFromSQL(context).update(query); + + auto collection = NamedCollectionFactory::instance().getMutable(query.collection_name); + auto collection_lock = collection->lock(); + + for (const auto & [name, value] : query.changes) + collection->setOrUpdate(name, convertFieldToString(value)); + + for (const auto & key : query.delete_keys) + collection->remove(key); +} + +} + +} diff --git a/src/Storages/NamedCollections/NamedCollectionUtils.h b/src/Storages/NamedCollections/NamedCollectionUtils.h new file mode 100644 index 00000000000..8befc9cac3c --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionUtils.h @@ -0,0 +1,40 @@ +#pragma once +#include + +namespace Poco { namespace Util { class AbstractConfiguration; } } + +namespace DB +{ + +class ASTCreateNamedCollectionQuery; +class ASTAlterNamedCollectionQuery; + +namespace NamedCollectionUtils +{ + +enum class SourceId +{ + NONE = 0, + CONFIG = 1, + SQL = 2, +}; + +void loadFromConfig(const Poco::Util::AbstractConfiguration & config); +void reloadFromConfig(const Poco::Util::AbstractConfiguration & config); + +/// Load named collections from `context->getPath() / named_collections /`. +void loadFromSQL(ContextPtr context); + +/// Remove collection as well as its metadata from `context->getPath() / named_collections /`. +void removeFromSQL(const std::string & collection_name, ContextPtr context); +void removeIfExistsFromSQL(const std::string & collection_name, ContextPtr context); + +/// Create a new collection from AST and put it to `context->getPath() / named_collections /`. +void createFromSQL(const ASTCreateNamedCollectionQuery & query, ContextPtr context); + +/// Update definition of already existing collection from AST and update result in `context->getPath() / named_collections /`. +void updateFromSQL(const ASTAlterNamedCollectionQuery & query, ContextPtr context); + +} + +} diff --git a/src/Storages/NamedCollections/NamedCollections.cpp b/src/Storages/NamedCollections/NamedCollections.cpp new file mode 100644 index 00000000000..03633bbd370 --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollections.cpp @@ -0,0 +1,450 @@ +#include "NamedCollections.h" + +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NAMED_COLLECTION_DOESNT_EXIST; + extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int NAMED_COLLECTION_IS_IMMUTABLE; +} + +namespace Configuration = NamedCollectionConfiguration; + + +NamedCollectionFactory & NamedCollectionFactory::instance() +{ + static NamedCollectionFactory instance; + return instance; +} + +bool NamedCollectionFactory::exists(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + return existsUnlocked(collection_name, lock); +} + +bool NamedCollectionFactory::existsUnlocked( + const std::string & collection_name, + std::lock_guard & /* lock */) const +{ + return loaded_named_collections.contains(collection_name); +} + +NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } + return collection; +} + +NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + return tryGetUnlocked(collection_name, lock); +} + +MutableNamedCollectionPtr NamedCollectionFactory::getMutable( + const std::string & collection_name) const +{ + std::lock_guard lock(mutex); + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } + else if (!collection->isMutable()) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, + "Cannot get collection `{}` for modification, " + "because collection was defined as immutable", + collection_name); + } + return collection; +} + +MutableNamedCollectionPtr NamedCollectionFactory::tryGetUnlocked( + const std::string & collection_name, + std::lock_guard & /* lock */) const +{ + auto it = loaded_named_collections.find(collection_name); + if (it == loaded_named_collections.end()) + return nullptr; + return it->second; +} + +void NamedCollectionFactory::add( + const std::string & collection_name, + MutableNamedCollectionPtr collection) +{ + std::lock_guard lock(mutex); + return addUnlocked(collection_name, collection, lock); +} + +void NamedCollectionFactory::add(NamedCollectionsMap collections) +{ + std::lock_guard lock(mutex); + for (const auto & [collection_name, collection] : collections) + addUnlocked(collection_name, collection, lock); +} + +void NamedCollectionFactory::addUnlocked( + const std::string & collection_name, + MutableNamedCollectionPtr collection, + std::lock_guard & /* lock */) +{ + auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); + if (!inserted) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + collection_name); + } +} + +void NamedCollectionFactory::remove(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + bool removed = removeIfExistsUnlocked(collection_name, lock); + if (!removed) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_DOESNT_EXIST, + "There is no named collection `{}`", + collection_name); + } +} + +void NamedCollectionFactory::removeIfExists(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + removeIfExistsUnlocked(collection_name, lock); +} + +bool NamedCollectionFactory::removeIfExistsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) +{ + auto collection = tryGetUnlocked(collection_name, lock); + if (!collection) + return false; + + if (!collection->isMutable()) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, + "Cannot get collection `{}` for modification, " + "because collection was defined as immutable", + collection_name); + } + loaded_named_collections.erase(collection_name); + return true; +} + +void NamedCollectionFactory::removeById(NamedCollectionUtils::SourceId id) +{ + std::lock_guard lock(mutex); + std::erase_if( + loaded_named_collections, + [&](const auto & value) { return value.second->getSourceId() == id; }); +} + +NamedCollectionsMap NamedCollectionFactory::getAll() const +{ + std::lock_guard lock(mutex); + return loaded_named_collections; +} + +class NamedCollection::Impl +{ +private: + ConfigurationPtr config; + Keys keys; + + Impl(ConfigurationPtr config_, const Keys & keys_) : config(config_) , keys(keys_) {} + +public: + static ImplPtr create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name, + const std::string & collection_path, + const Keys & keys) + { + auto collection_config = NamedCollectionConfiguration::createEmptyConfiguration(collection_name); + for (const auto & key : keys) + Configuration::copyConfigValue( + config, collection_path + '.' + key, *collection_config, key); + + return std::unique_ptr(new Impl(collection_config, keys)); + } + + template T get(const Key & key) const + { + return Configuration::getConfigValue(*config, key); + } + + template T getOrDefault(const Key & key, const T & default_value) const + { + return Configuration::getConfigValueOrDefault(*config, key, &default_value); + } + + template void set(const Key & key, const T & value, bool update_if_exists) + { + Configuration::setConfigValue(*config, key, value, update_if_exists); + if (!keys.contains(key)) + keys.insert(key); + } + + ImplPtr createCopy(const std::string & collection_name_) const + { + return create(*config, collection_name_, "", keys); + } + + void remove(const Key & key) + { + Configuration::removeConfigValue(*config, key); + [[maybe_unused]] auto removed = keys.erase(key); + assert(removed); + } + + Keys getKeys() const + { + return keys; + } + + Keys::const_iterator begin() const + { + return keys.begin(); + } + + Keys::const_iterator end() const + { + return keys.end(); + } + + std::string dumpStructure() const + { + /// Convert a collection config like + /// + /// value0 + /// + /// value2 + /// + /// value3 + /// + /// + /// + /// to a string: + /// "key0: value0 + /// key1: + /// key2: value2 + /// key3: + /// key4: value3" + WriteBufferFromOwnString wb; + Strings prev_key_parts; + for (const auto & key : keys) + { + Strings key_parts; + splitInto<'.'>(key_parts, key); + size_t tab_cnt = 0; + + auto it = key_parts.begin(); + auto prev_key_parts_it = prev_key_parts.begin(); + while (it != key_parts.end() + && prev_key_parts_it != prev_key_parts.end() + && *it == *prev_key_parts_it) + { + ++it; + ++prev_key_parts_it; + ++tab_cnt; + } + + auto start_it = it; + for (; it != key_parts.end(); ++it) + { + if (it != start_it) + wb << '\n'; + wb << std::string(tab_cnt++, '\t'); + wb << *it << ':'; + } + wb << '\t' << get(key) << '\n'; + prev_key_parts = key_parts; + } + return wb.str(); + } +}; + +NamedCollection::NamedCollection( + ImplPtr pimpl_, + const std::string & collection_name_, + SourceId source_id_, + bool is_mutable_) + : pimpl(std::move(pimpl_)) + , collection_name(collection_name_) + , source_id(source_id_) + , is_mutable(is_mutable_) +{ +} + +MutableNamedCollectionPtr NamedCollection::create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name, + const std::string & collection_path, + const Keys & keys, + SourceId source_id, + bool is_mutable) +{ + auto impl = Impl::create(config, collection_name, collection_path, keys); + return std::unique_ptr( + new NamedCollection(std::move(impl), collection_name, source_id, is_mutable)); +} + +template T NamedCollection::get(const Key & key) const +{ + std::lock_guard lock(mutex); + return pimpl->get(key); +} + +template T NamedCollection::getOrDefault(const Key & key, const T & default_value) const +{ + std::lock_guard lock(mutex); + return pimpl->getOrDefault(key, default_value); +} + +template void NamedCollection::set(const Key & key, const T & value) +{ + assertMutable(); + std::unique_lock lock(mutex, std::defer_lock); + if constexpr (!Locked) + lock.lock(); + pimpl->set(key, value, false); +} + +template void NamedCollection::setOrUpdate(const Key & key, const T & value) +{ + assertMutable(); + std::unique_lock lock(mutex, std::defer_lock); + if constexpr (!Locked) + lock.lock(); + pimpl->set(key, value, true); +} + +template void NamedCollection::remove(const Key & key) +{ + assertMutable(); + std::unique_lock lock(mutex, std::defer_lock); + if constexpr (!Locked) + lock.lock(); + pimpl->remove(key); +} + +void NamedCollection::assertMutable() const +{ + if (!is_mutable) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_IS_IMMUTABLE, + "Cannot change named collection because it is immutable"); + } +} + +MutableNamedCollectionPtr NamedCollection::duplicate() const +{ + std::lock_guard lock(mutex); + auto impl = pimpl->createCopy(collection_name); + return std::unique_ptr( + new NamedCollection( + std::move(impl), collection_name, NamedCollectionUtils::SourceId::NONE, true)); +} + +NamedCollection::Keys NamedCollection::getKeys() const +{ + std::lock_guard lock(mutex); + return pimpl->getKeys(); +} + +template NamedCollection::const_iterator NamedCollection::begin() const +{ + std::unique_lock lock(mutex, std::defer_lock); + if constexpr (!Locked) + lock.lock(); + return pimpl->begin(); +} + +template NamedCollection::const_iterator NamedCollection::end() const +{ + std::unique_lock lock(mutex, std::defer_lock); + if constexpr (!Locked) + lock.lock(); + return pimpl->end(); +} + +std::string NamedCollection::dumpStructure() const +{ + std::lock_guard lock(mutex); + return pimpl->dumpStructure(); +} + +std::unique_lock NamedCollection::lock() +{ + return std::unique_lock(mutex); +} + +template String NamedCollection::get(const NamedCollection::Key & key) const; +template UInt64 NamedCollection::get(const NamedCollection::Key & key) const; +template Int64 NamedCollection::get(const NamedCollection::Key & key) const; +template Float64 NamedCollection::get(const NamedCollection::Key & key) const; + +template String NamedCollection::getOrDefault(const NamedCollection::Key & key, const String & default_value) const; +template UInt64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const UInt64 & default_value) const; +template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Int64 & default_value) const; +template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; + +template void NamedCollection::set(const NamedCollection::Key & key, const String & value); +template void NamedCollection::set(const NamedCollection::Key & key, const String & value); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); + +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const String & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const String & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const Float64 & value); +template void NamedCollection::setOrUpdate(const NamedCollection::Key & key, const Float64 & value); + +template void NamedCollection::remove(const Key & key); +template void NamedCollection::remove(const Key & key); + +template NamedCollection::const_iterator NamedCollection::begin() const; +template NamedCollection::const_iterator NamedCollection::begin() const; +template NamedCollection::const_iterator NamedCollection::end() const; +template NamedCollection::const_iterator NamedCollection::end() const; +} diff --git a/src/Storages/NamedCollections/NamedCollections.h b/src/Storages/NamedCollections/NamedCollections.h new file mode 100644 index 00000000000..53b7a828a63 --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollections.h @@ -0,0 +1,139 @@ +#pragma once +#include +#include +#include + +namespace Poco { namespace Util { class AbstractConfiguration; } } + +namespace DB +{ + +/** + * Class to represent arbitrary-structured named collection object. + * It can be defined via config or via SQL command. + * + * + * ... + * + * ... + * + */ +class NamedCollection +{ +public: + using Key = std::string; + using Keys = std::set>; + using SourceId = NamedCollectionUtils::SourceId; + + static MutableNamedCollectionPtr create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name, + const std::string & collection_path, + const Keys & keys, + SourceId source_id_, + bool is_mutable_); + + template T get(const Key & key) const; + + template T getOrDefault(const Key & key, const T & default_value) const; + + std::unique_lock lock(); + + template void set(const Key & key, const T & value); + + template void setOrUpdate(const Key & key, const T & value); + + template void remove(const Key & key); + + MutableNamedCollectionPtr duplicate() const; + + Keys getKeys() const; + + using iterator = typename Keys::iterator; + using const_iterator = typename Keys::const_iterator; + + template const_iterator begin() const; + + template const_iterator end() const; + + std::string dumpStructure() const; + + bool isMutable() const { return is_mutable; } + + SourceId getSourceId() const { return source_id; } + +private: + class Impl; + using ImplPtr = std::unique_ptr; + + NamedCollection( + ImplPtr pimpl_, + const std::string & collection_name, + SourceId source_id, + bool is_mutable); + + void assertMutable() const; + + ImplPtr pimpl; + const std::string collection_name; + const SourceId source_id; + const bool is_mutable; + mutable std::mutex mutex; +}; + +/** + * A factory of immutable named collections. + */ +class NamedCollectionFactory : boost::noncopyable +{ +public: + static NamedCollectionFactory & instance(); + + bool exists(const std::string & collection_name) const; + + NamedCollectionPtr get(const std::string & collection_name) const; + + NamedCollectionPtr tryGet(const std::string & collection_name) const; + + MutableNamedCollectionPtr getMutable(const std::string & collection_name) const; + + void add(const std::string & collection_name, MutableNamedCollectionPtr collection); + + void add(NamedCollectionsMap collections); + + void update(NamedCollectionsMap collections); + + void remove(const std::string & collection_name); + + void removeIfExists(const std::string & collection_name); + + void removeById(NamedCollectionUtils::SourceId id); + + NamedCollectionsMap getAll() const; + +private: + bool existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + + MutableNamedCollectionPtr tryGetUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + + void addUnlocked( + const std::string & collection_name, + MutableNamedCollectionPtr collection, + std::lock_guard & lock); + + bool removeIfExistsUnlocked( + const std::string & collection_name, + std::lock_guard & lock); + + mutable NamedCollectionsMap loaded_named_collections; + + mutable std::mutex mutex; + bool is_initialized = false; +}; + + +} diff --git a/src/Storages/NamedCollections/NamedCollectionsHelpers.cpp b/src/Storages/NamedCollections/NamedCollectionsHelpers.cpp new file mode 100644 index 00000000000..a2fbcd05619 --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionsHelpers.cpp @@ -0,0 +1,107 @@ +#include "NamedCollectionsHelpers.h" +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + NamedCollectionPtr tryGetNamedCollectionFromASTs(ASTs asts) + { + if (asts.empty()) + return nullptr; + + const auto * identifier = asts[0]->as(); + if (!identifier) + return nullptr; + + const auto & collection_name = identifier->name(); + return NamedCollectionFactory::instance().tryGet(collection_name); + } + + std::optional> getKeyValueFromAST(ASTPtr ast) + { + const auto * function = ast->as(); + if (!function || function->name != "equals") + return std::nullopt; + + const auto * function_args_expr = assert_cast(function->arguments.get()); + const auto & function_args = function_args_expr->children; + + if (function_args.size() != 2) + return std::nullopt; + + auto literal_key = evaluateConstantExpressionOrIdentifierAsLiteral( + function_args[0], Context::getGlobalContextInstance()); + auto key = checkAndGetLiteralArgument(literal_key, "key"); + + auto literal_value = evaluateConstantExpressionOrIdentifierAsLiteral( + function_args[1], Context::getGlobalContextInstance()); + auto value = literal_value->as()->value; + + return std::pair{key, value}; + } +} + + +NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts) +{ + if (asts.empty()) + return nullptr; + + auto collection = tryGetNamedCollectionFromASTs(asts); + if (!collection) + return nullptr; + + if (asts.size() == 1) + return collection; + + auto collection_copy = collection->duplicate(); + + for (const auto & ast : asts) + { + auto value_override = getKeyValueFromAST(ast); + if (!value_override) + continue; + + const auto & [key, value] = *value_override; + collection_copy->set(key, toString(value)); + } + + return collection_copy; +} + +void validateNamedCollection( + const NamedCollection & collection, + const std::unordered_set & required_keys, + const std::unordered_set & optional_keys) +{ + const auto & keys = collection.getKeys(); + for (const auto & key : keys) + { + if (!required_keys.contains(key) && !optional_keys.contains(key)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unexpected key `{}` in named collection. Required keys: {}, optional keys: {}", + key, fmt::join(required_keys, ", "), fmt::join(optional_keys, ", ")); + } + } + + for (const auto & key : required_keys) + { + if (!keys.contains(key)) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Key `{}` is required, but not specified. Required keys: {}, optional keys: {}", + key, fmt::join(required_keys, ", "), fmt::join(optional_keys, ", ")); + } + } +} + +} diff --git a/src/Storages/NamedCollections/NamedCollectionsHelpers.h b/src/Storages/NamedCollections/NamedCollectionsHelpers.h new file mode 100644 index 00000000000..39baafa9039 --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollectionsHelpers.h @@ -0,0 +1,18 @@ +#pragma once +#include +#include +#include +#include + + +namespace DB +{ + +NamedCollectionPtr tryGetNamedCollectionWithOverrides(ASTs asts); + +void validateNamedCollection( + const NamedCollection & collection, + const std::unordered_set & required_keys, + const std::unordered_set & optional_keys); + +} diff --git a/src/Storages/NamedCollections/NamedCollections_fwd.h b/src/Storages/NamedCollections/NamedCollections_fwd.h new file mode 100644 index 00000000000..47ebe81c91f --- /dev/null +++ b/src/Storages/NamedCollections/NamedCollections_fwd.h @@ -0,0 +1,12 @@ +#pragma once +#include + +namespace DB +{ + +class NamedCollection; +using NamedCollectionPtr = std::shared_ptr; +using MutableNamedCollectionPtr = std::shared_ptr; +using NamedCollectionsMap = std::map; + +} diff --git a/src/Storages/StorageConfiguration.h b/src/Storages/StorageConfiguration.h new file mode 100644 index 00000000000..cab95203c81 --- /dev/null +++ b/src/Storages/StorageConfiguration.h @@ -0,0 +1,15 @@ +#pragma once +#include + +namespace DB +{ + +/// A base class for stateless table engines configurations. +struct StatelessTableEngineConfiguration +{ + String format = "auto"; + String compression_method = "auto"; + String structure = "auto"; +}; + +} diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 24602a1e114..93474cc1753 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -32,6 +32,8 @@ #include #include #include +#include +#include #include #include @@ -66,8 +68,6 @@ namespace fs = std::filesystem; -static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - namespace ProfileEvents { extern const Event S3DeleteObjects; @@ -77,6 +77,27 @@ namespace ProfileEvents namespace DB { +static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + +static const std::unordered_set required_configuration_keys = { + "url", +}; +static std::unordered_set optional_configuration_keys = { + "format", + "compression", + "structure", + "access_key_id", + "secret_access_key", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", +}; + namespace ErrorCodes { extern const int CANNOT_PARSE_TEXT; @@ -230,7 +251,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext .last_modification_time = row.GetLastModified().Millis() / 1000, }; - if (object_infos) + if (object_infos != nullptr) (*object_infos)[fs::path(globbed_uri.bucket) / key] = info; temp_buffer.emplace_back(std::move(key), std::move(info)); @@ -275,7 +296,7 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext /// Set iterator only after the whole batch is processed buffer_iter = buffer.begin(); - if (read_keys) + if (read_keys != nullptr) { read_keys->reserve(read_keys->size() + buffer.size()); for (const auto & [key, _] : buffer) @@ -411,7 +432,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext } } - if (read_keys_) + if (read_keys_ != nullptr) *read_keys_ = all_keys; for (auto && key : all_keys) @@ -420,7 +441,7 @@ class StorageS3Source::KeysIterator::Impl : WithContext /// To avoid extra requests update total_size only if object_infos != nullptr /// (which means we eventually need this info anyway, so it should be ok to do it now) - if (object_infos_) + if (object_infos_ != nullptr) { info = S3::getObjectInfo(client_, bucket, key, version_id_, true, false); total_size += info->size; @@ -502,7 +523,7 @@ StorageS3Source::StorageS3Source( const ColumnsDescription & columns_, UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, - const String compression_hint_, + const String & compression_hint_, const std::shared_ptr & client_, const String & bucket_, const String & version_id_, @@ -673,7 +694,7 @@ Chunk StorageS3Source::generate() const auto & file_path = reader.getPath(); size_t total_size = file_iterator->getTotalSize(); - if (num_rows && total_size) + if ((num_rows != 0u) && (total_size != 0u)) { updateRowsProgressApprox( *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); @@ -865,7 +886,7 @@ class PartitionedStorageS3Sink : public PartitionedSink { S3::URI::validateBucket(str, {}); - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + if (DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size()) == 0u) throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name"); validatePartitionKey(str, false); @@ -880,7 +901,7 @@ class PartitionedStorageS3Sink : public PartitionedSink if (str.empty() || str.size() > 1024) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size()); - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + if (DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size()) == 0u) throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key"); validatePartitionKey(str, true); @@ -1168,10 +1189,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - if (upd.request_settings != settings.request_settings) - upd.request_settings = settings.request_settings; - - upd.request_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + upd.request_settings = settings.request_settings; if (upd.client) { @@ -1212,48 +1230,43 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration upd.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); } - -void StorageS3::processNamedCollectionResult(StorageS3Configuration & configuration, const std::vector> & key_value_args) +void StorageS3::processNamedCollectionResult(StorageS3Configuration & configuration, const NamedCollection & collection) { - for (const auto & [arg_name, arg_value] : key_value_args) + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + std::string filename; + + configuration.request_settings = S3Settings::RequestSettings(collection); + + for (const auto & key : collection) { - if (arg_name == "access_key_id") - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(arg_value, "access_key_id"); - else if (arg_name == "secret_access_key") - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(arg_value, "secret_access_key"); - else if (arg_name == "filename") - configuration.url = std::filesystem::path(configuration.url) / checkAndGetLiteralArgument(arg_value, "filename"); - else if (arg_name == "use_environment_credentials") - configuration.auth_settings.use_environment_credentials = checkAndGetLiteralArgument(arg_value, "use_environment_credentials"); - else if (arg_name == "max_single_read_retries") - configuration.request_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); - else if (arg_name == "min_upload_part_size") - configuration.request_settings.min_upload_part_size = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); - else if (arg_name == "upload_part_size_multiply_factor") - configuration.request_settings.upload_part_size_multiply_factor = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); - else if (arg_name == "upload_part_size_multiply_parts_count_threshold") - configuration.request_settings.upload_part_size_multiply_parts_count_threshold = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); - else if (arg_name == "max_single_part_upload_size") - configuration.request_settings.max_single_part_upload_size = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); - else if (arg_name == "max_connections") - configuration.request_settings.max_connections = checkAndGetLiteralArgument(arg_value, "max_connections"); - else - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].", - arg_name); + if (key == "url") + configuration.url = collection.get(key); + else if (key == "access_key_id") + configuration.auth_settings.access_key_id = collection.get(key); + else if (key == "secret_access_key") + configuration.auth_settings.secret_access_key = collection.get(key); + else if (key == "filename") + filename = collection.get(key); + else if (key == "format") + configuration.format = collection.get(key); + else if (key == "compression") + configuration.compression_method = collection.get(key); + else if (key == "structure") + configuration.structure = collection.get(key); + else if (key == "use_environment_credentials") + configuration.auth_settings.use_environment_credentials = collection.get(key); } + if (!filename.empty()) + configuration.url = std::filesystem::path(configuration.url) / filename; } - StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context) { StorageS3Configuration configuration; - if (auto named_collection = getURLBasedDataSourceConfiguration(engine_args, local_context)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); - configuration.set(common_configuration); - processNamedCollectionResult(configuration, storage_specific_args); + processNamedCollectionResult(configuration, *named_collection); } else { diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 9d0a50ca66e..503c3d1bd5b 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -34,6 +35,8 @@ namespace DB class PullingPipelineExecutor; class StorageS3SequentialSource; +class NamedCollection; + class StorageS3Source final : public ISource, WithContext { public: @@ -133,7 +136,7 @@ class StorageS3Source final : public ISource, WithContext const ColumnsDescription & columns_, UInt64 max_block_size_, const S3Settings::RequestSettings & request_settings_, - const String compression_hint_, + const String & compression_hint_, const std::shared_ptr & client_, const String & bucket, const String & version_id, @@ -268,7 +271,7 @@ class StorageS3 : public shared_ptr_helper, public IStorage, WithCont ContextPtr ctx, ObjectInfos * object_infos = nullptr); - static void processNamedCollectionResult(StorageS3Configuration & configuration, const std::vector> & key_value_args); + static void processNamedCollectionResult(StorageS3Configuration & configuration, const NamedCollection & collection); struct S3Configuration { diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 962f7c113bd..9f322d196ee 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include #include @@ -13,69 +14,192 @@ namespace DB { -namespace +namespace ErrorCodes { - /// An object up to 5 GB can be copied in a single atomic operation. - constexpr UInt64 DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE = 5_GiB; + extern const int INVALID_SETTING_VALUE; +} + +S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings) + : PartUploadSettings() +{ + min_upload_part_size = settings.s3_min_upload_part_size; + upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; + upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; + max_single_part_upload_size = settings.s3_max_single_part_upload_size; - /// The maximum size of an uploaded part. - constexpr UInt64 DEFAULT_MAX_UPLOAD_PART_SIZE = 5_GiB; + validate(); } +S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedCollection & collection) + : PartUploadSettings() +{ + min_upload_part_size = collection.getOrDefault("min_upload_part_size", min_upload_part_size); + upload_part_size_multiply_factor = collection.getOrDefault("upload_part_size_multiply_factor", upload_part_size_multiply_factor); + upload_part_size_multiply_parts_count_threshold = collection.getOrDefault("upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); + max_single_part_upload_size = collection.getOrDefault("max_single_part_upload_size", max_single_part_upload_size); -void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) + validate(); +} + +S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( + const Poco::Util::AbstractConfiguration & config, + const String & key, + const Settings & settings) + : PartUploadSettings(settings) { - std::lock_guard lock(mutex); - s3_settings.clear(); - if (!config.has(config_elem)) - return; + min_upload_part_size = config.getUInt64(key + ".min_upload_part_size", min_upload_part_size); + max_upload_part_size = config.getUInt64(key + ".max_upload_part_size", max_upload_part_size); + upload_part_size_multiply_factor = config.getUInt64(key + ".upload_part_size_multiply_factor", upload_part_size_multiply_factor); + upload_part_size_multiply_parts_count_threshold = config.getUInt64(key + ".upload_part_size_multiply_parts_count_threshold", upload_part_size_multiply_parts_count_threshold); + max_part_number = config.getUInt64(key + ".max_part_number", max_part_number); + max_single_part_upload_size = config.getUInt64(key + ".max_single_part_upload_size", max_single_part_upload_size); + max_single_operation_copy_size = config.getUInt64(key + ".max_single_operation_copy_size", max_single_operation_copy_size); - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(config_elem, config_keys); + validate(); +} - auto get_string_for_key = [&](const String & key, const String & elem, bool with_default = true, const String & default_value = "") - { - return with_default ? config.getString(config_elem + "." + key + "." + elem, default_value) : config.getString(config_elem + "." + key + "." + elem); - }; +void S3Settings::RequestSettings::PartUploadSettings::validate() +{ + static constexpr size_t min_upload_part_size_limit = 5 * 1024 * 1024; + if (min_upload_part_size < min_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting min_upload_part_size has invalid value {} which is less than the s3 API limit {}", + ReadableSize(min_upload_part_size), ReadableSize(min_upload_part_size_limit)); + + static constexpr size_t max_upload_part_size_limit = 5ull * 1024 * 1024 * 1024; + if (max_upload_part_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_upload_part_size has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); + + if (max_single_part_upload_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_single_part_upload_size has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_single_part_upload_size), ReadableSize(max_upload_part_size_limit)); + + if (max_single_operation_copy_size > max_upload_part_size_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_single_operation_copy_size has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_single_operation_copy_size), ReadableSize(max_upload_part_size_limit)); + + if (max_upload_part_size < min_upload_part_size) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_upload_part_size ({}) can't be less than setting min_upload_part_size {}", + ReadableSize(max_upload_part_size), ReadableSize(min_upload_part_size)); + + if (!upload_part_size_multiply_factor) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_factor cannot be zero", + upload_part_size_multiply_factor); + + if (!upload_part_size_multiply_parts_count_threshold) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_parts_count_threshold cannot be zero", + upload_part_size_multiply_parts_count_threshold); + + if (!max_part_number) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_part_number cannot be zero", + max_part_number); + + static constexpr size_t max_part_number_limit = 10000; + if (max_part_number > max_part_number_limit) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting max_part_number has invalid value {} which is grater than the s3 API limit {}", + ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); + + size_t maybe_overflow; + if (common::mulOverflow(max_upload_part_size, upload_part_size_multiply_factor, maybe_overflow)) + throw Exception( + ErrorCodes::INVALID_SETTING_VALUE, + "Setting upload_part_size_multiply_factor is too big ({}). Multiplication to max_upload_part_size ({}) will cause integer overflow", + ReadableSize(max_part_number), ReadableSize(max_part_number_limit)); + + /// TODO: it's possible to set too small limits. We can check that max possible object size is not too small. +} + +S3Settings::RequestSettings::RequestSettings(const Settings & settings) + : upload_settings(settings) +{ + max_single_read_retries = settings.s3_max_single_read_retries; + max_connections = settings.s3_max_connections; + check_objects_after_upload = settings.s3_check_objects_after_upload; + max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; + if (settings.s3_max_get_rps) + get_request_throttler = std::make_shared( + settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); + if (settings.s3_max_put_rps) + put_request_throttler = std::make_shared( + settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); +} - auto get_uint_for_key = [&](const String & key, const String & elem, bool with_default = true, UInt64 default_value = 0) +S3Settings::RequestSettings::RequestSettings(const NamedCollection & collection) + : upload_settings(collection) +{ + max_single_read_retries = collection.getOrDefault("max_single_read_retries", max_single_read_retries); + max_connections = collection.getOrDefault("max_connections", max_connections); +} + +S3Settings::RequestSettings::RequestSettings( + const Poco::Util::AbstractConfiguration & config, + const String & key, + const Settings & settings) + : upload_settings(config, key, settings) +{ + max_single_read_retries = config.getUInt64(key + ".max_single_read_retries", settings.s3_max_single_read_retries); + max_connections = config.getUInt64(key + ".max_connections", settings.s3_max_connections); + check_objects_after_upload = config.getBool(key + ".check_objects_after_upload", false); + + /// NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, + /// which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. + if (UInt64 max_get_rps = config.getUInt64(key + ".max_get_rps", settings.s3_max_get_rps)) { - return with_default ? config.getUInt64(config_elem + "." + key + "." + elem, default_value) : config.getUInt64(config_elem + "." + key + "." + elem); - }; + size_t default_max_get_burst = settings.s3_max_get_burst + ? settings.s3_max_get_burst + : (Throttler::default_burst_seconds * max_get_rps); + size_t max_get_burst = config.getUInt64(key + ".max_get_burst", default_max_get_burst); - auto get_bool_for_key = [&](const String & key, const String & elem, bool with_default = true, bool default_value = false) + get_request_throttler = std::make_shared(max_get_rps, max_get_burst); + } + if (UInt64 max_put_rps = config.getUInt64(key + ".max_put_rps", settings.s3_max_put_rps)) { - return with_default ? config.getBool(config_elem + "." + key + "." + elem, default_value) : config.getBool(config_elem + "." + key + "." + elem); - }; + size_t default_max_put_burst = settings.s3_max_put_burst + ? settings.s3_max_put_burst + : (Throttler::default_burst_seconds * max_put_rps); + + size_t max_put_burst = config.getUInt64(key + ".max_put_burst", default_max_put_burst); + + put_request_throttler = std::make_shared(max_put_rps, max_put_burst); + } +} + +void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) +{ + std::lock_guard lock(mutex); + s3_settings.clear(); + if (!config.has(config_elem)) + return; + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(config_elem, config_keys); for (const String & key : config_keys) { if (config.has(config_elem + "." + key + ".endpoint")) { - auto endpoint = get_string_for_key(key, "endpoint", false); - + auto endpoint = config.getString(config_elem + "." + key + ".endpoint"); auto auth_settings = S3::AuthSettings::loadFromConfig(config_elem + "." + key, config); - - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = get_uint_for_key(key, "max_single_read_retries", true, settings.s3_max_single_read_retries); - request_settings.min_upload_part_size = get_uint_for_key(key, "min_upload_part_size", true, settings.s3_min_upload_part_size); - request_settings.max_upload_part_size = get_uint_for_key(key, "max_upload_part_size", true, DEFAULT_MAX_UPLOAD_PART_SIZE); - request_settings.upload_part_size_multiply_factor = get_uint_for_key(key, "upload_part_size_multiply_factor", true, settings.s3_upload_part_size_multiply_factor); - request_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); - request_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); - request_settings.max_single_operation_copy_size = get_uint_for_key(key, "max_single_operation_copy_size", true, DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE); - request_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); - request_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); - - // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = get_uint_for_key(key, "max_get_rps", true, settings.s3_max_get_rps)) - request_settings.get_request_throttler = std::make_shared( - max_get_rps, get_uint_for_key(key, "max_get_burst", true, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); - if (UInt64 max_put_rps = get_uint_for_key(key, "max_put_rps", true, settings.s3_max_put_rps)) - request_settings.put_request_throttler = std::make_shared( - max_put_rps, get_uint_for_key(key, "max_put_burst", true, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); + S3Settings::RequestSettings request_settings(config, config_elem + "." + key, settings); s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); } @@ -98,51 +222,36 @@ S3Settings StorageS3Settings::getSettings(const String & endpoint) const return {}; } -S3Settings::RequestSettings::RequestSettings(const Settings & settings) -{ - max_single_read_retries = settings.s3_max_single_read_retries; - min_upload_part_size = settings.s3_min_upload_part_size; - upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; - upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; - max_single_part_upload_size = settings.s3_max_single_part_upload_size; - max_connections = settings.s3_max_connections; - check_objects_after_upload = settings.s3_check_objects_after_upload; - max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - if (settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - if (settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); -} -void S3Settings::RequestSettings::updateFromSettingsIfEmpty(const Settings & settings) -{ - if (!max_single_read_retries) - max_single_read_retries = settings.s3_max_single_read_retries; - if (!min_upload_part_size) - min_upload_part_size = settings.s3_min_upload_part_size; - if (!max_upload_part_size) - max_upload_part_size = DEFAULT_MAX_UPLOAD_PART_SIZE; - if (!upload_part_size_multiply_factor) - upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; - if (!upload_part_size_multiply_parts_count_threshold) - upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; - if (!max_single_part_upload_size) - max_single_part_upload_size = settings.s3_max_single_part_upload_size; - if (!max_single_operation_copy_size) - max_single_operation_copy_size = DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE; - if (!max_connections) - max_connections = settings.s3_max_connections; - if (!max_unexpected_write_error_retries) - max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - check_objects_after_upload = settings.s3_check_objects_after_upload; - if (!get_request_throttler && settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - if (!put_request_throttler && settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); -} +// void S3Settings::RequestSettings::updateFromSettingsIfEmpty(const Settings & settings) +// { +// if (!max_single_read_retries) +// max_single_read_retries = settings.s3_max_single_read_retries; +// if (!min_upload_part_size) +// min_upload_part_size = settings.s3_min_upload_part_size; +// if (!max_upload_part_size) +// max_upload_part_size = DEFAULT_MAX_UPLOAD_PART_SIZE; +// if (!upload_part_size_multiply_factor) +// upload_part_size_multiply_factor = settings.s3_upload_part_size_multiply_factor; +// if (!upload_part_size_multiply_parts_count_threshold) +// upload_part_size_multiply_parts_count_threshold = settings.s3_upload_part_size_multiply_parts_count_threshold; +// if (!max_part_number) +// max_part_number = DEFAULT_MAX_PART_NUMBER; +// if (!max_single_part_upload_size) +// max_single_part_upload_size = settings.s3_max_single_part_upload_size; +// if (!max_single_operation_copy_size) +// max_single_operation_copy_size = DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE; +// if (!max_connections) +// max_connections = settings.s3_max_connections; +// if (!max_unexpected_write_error_retries) +// max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; +// check_objects_after_upload = settings.s3_check_objects_after_upload; +// if (!get_request_throttler && settings.s3_max_get_rps) +// get_request_throttler = std::make_shared( +// settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); +// if (!put_request_throttler && settings.s3_max_put_rps) +// put_request_throttler = std::make_shared( +// settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); +// } } diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 955cd2d025b..42b840dec3e 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -21,36 +21,61 @@ namespace DB { struct Settings; +class NamedCollection; struct S3Settings { struct RequestSettings { - size_t max_single_read_retries = 0; - size_t min_upload_part_size = 0; - size_t max_upload_part_size = 0; - size_t upload_part_size_multiply_factor = 0; - size_t upload_part_size_multiply_parts_count_threshold = 0; - size_t max_single_part_upload_size = 0; - size_t max_single_operation_copy_size = 0; - size_t max_connections = 0; + struct PartUploadSettings + { + size_t min_upload_part_size = 16 * 1024 * 1024; + size_t max_upload_part_size = 5ULL * 1024 * 1024 * 1024; + size_t upload_part_size_multiply_factor = 2; + size_t upload_part_size_multiply_parts_count_threshold = 500; + size_t max_part_number = 10000; + size_t max_single_part_upload_size = 32 * 1024 * 1024; + size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; + + inline bool operator==(const PartUploadSettings & other) const + { + return min_upload_part_size == other.min_upload_part_size + && max_upload_part_size == other.max_upload_part_size + && upload_part_size_multiply_factor == other.upload_part_size_multiply_factor + && upload_part_size_multiply_parts_count_threshold == other.upload_part_size_multiply_parts_count_threshold + && max_part_number == other.max_part_number + && max_single_part_upload_size == other.max_single_part_upload_size + && max_single_operation_copy_size == other.max_single_operation_copy_size; + } + + private: + PartUploadSettings() = default; + explicit PartUploadSettings(const Settings & settings); + explicit PartUploadSettings(const NamedCollection & collection); + PartUploadSettings(const Poco::Util::AbstractConfiguration & config, const String & key, const Settings & settings); + + void validate(); + + friend struct RequestSettings; + }; + + private: + PartUploadSettings upload_settings = {}; + + public: + size_t max_single_read_retries = 4; + size_t max_connections = 1024; bool check_objects_after_upload = false; - size_t max_unexpected_write_error_retries = 0; + size_t max_unexpected_write_error_retries = 4; ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; - RequestSettings() = default; - explicit RequestSettings(const Settings & settings); + const PartUploadSettings & getUploadSettings() const { return upload_settings; } inline bool operator==(const RequestSettings & other) const { - return max_single_read_retries == other.max_single_read_retries - && min_upload_part_size == other.min_upload_part_size - && max_upload_part_size == other.max_upload_part_size - && upload_part_size_multiply_factor == other.upload_part_size_multiply_factor - && upload_part_size_multiply_parts_count_threshold == other.upload_part_size_multiply_parts_count_threshold - && max_single_part_upload_size == other.max_single_part_upload_size - && max_single_operation_copy_size == other.max_single_operation_copy_size + return upload_settings == other.upload_settings + && max_single_read_retries == other.max_single_read_retries && max_connections == other.max_connections && check_objects_after_upload == other.check_objects_after_upload && max_unexpected_write_error_retries == other.max_unexpected_write_error_retries @@ -58,7 +83,10 @@ struct S3Settings && put_request_throttler == other.put_request_throttler; } - void updateFromSettingsIfEmpty(const Settings & settings); + RequestSettings() = default; + explicit RequestSettings(const Settings & settings); + explicit RequestSettings(const NamedCollection & collection); + RequestSettings(const Poco::Util::AbstractConfiguration & config, const String & key, const Settings & settings); }; S3::AuthSettings auth_settings; diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 3fed8c7f2d6..c979bd1cd4c 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include "registerTableFunctions.h" @@ -29,11 +30,9 @@ namespace ErrorCodes /// This is needed to avoid copy-pase. Because s3Cluster arguments only differ in additional argument (first) - cluster name void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & s3_configuration) { - if (auto named_collection = getURLBasedDataSourceConfiguration(args, context)) + if (auto named_collection = tryGetNamedCollectionWithOverrides(args)) { - auto [common_configuration, storage_specific_args] = named_collection.value(); - s3_configuration.set(common_configuration); - StorageS3::processNamedCollectionResult(s3_configuration, storage_specific_args); + StorageS3::processNamedCollectionResult(s3_configuration, *named_collection); } else {