From 6f03ff20d88398c2c097d031b00c1407c6624ca3 Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 13 Jul 2024 14:06:24 +0000 Subject: [PATCH 01/33] Add local storage --- .../ObjectStorage/Local/Configuration.cpp | 75 +++++++++ .../ObjectStorage/Local/Configuration.h | 55 +++++++ .../test_local_storage/__init__.py | 0 tests/integration/test_local_storage/test.py | 153 ++++++++++++++++++ 4 files changed, 283 insertions(+) create mode 100644 src/Storages/ObjectStorage/Local/Configuration.cpp create mode 100644 src/Storages/ObjectStorage/Local/Configuration.h create mode 100644 tests/integration/test_local_storage/__init__.py create mode 100644 tests/integration/test_local_storage/test.py diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp new file mode 100644 index 00000000000..5b66dd2470b --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -0,0 +1,75 @@ +#include + +#include +#include +#include +#include "Common/NamedCollections/NamedCollections.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int LOGICAL_ERROR; +} + +void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) +{ + path = collection.get("path"); + format = collection.getOrDefault("format", "auto"); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); +} + + +void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + const size_t max_args_num = with_structure ? 4 : 3; + if (args.empty() || args.size() > max_args_num) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num); + } + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + path = checkAndGetLiteralArgument(args[0], "path"); + + if (args.size() > 1) + { + format = checkAndGetLiteralArgument(args[1], "format_name"); + } + + if (with_structure) + { + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + } + if (args.size() > 3) + { + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + } + } + else if (args.size() > 2) + { + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + } +} + +StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.engine_file_skip_empty_files, + .list_object_keys_size = 0, + .throw_on_zero_files_match = false, + .ignore_non_existent_file = false}; +} + +} diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h new file mode 100644 index 00000000000..a305fc43825 --- /dev/null +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include "Disks/ObjectStorages/Local/LocalObjectStorage.h" +#include "config.h" + +#if USE_AWS_S3 +# include +# include + +namespace DB +{ + +class StorageLocalConfiguration : public StorageObjectStorage::Configuration +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + static constexpr auto type_name = "local"; + + StorageLocalConfiguration() = default; + StorageLocalConfiguration(const StorageLocalConfiguration & other) = default; + + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return "Local"; } + + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + void setPaths(const Paths & paths_) override { paths = paths_; } + + String getNamespace() const override { return ""; } + String getDataSourceDescription() const override { return ""; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + + void check(ContextPtr) const override { } + void validateNamespace(const String &) const override { } + ConfigurationPtr clone() override { return std::make_shared(*this); } + bool isStaticConfiguration() const override { return true; } + + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + + void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } + +private: + void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + Path path; + Paths paths; +}; + +} + +#endif diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py new file mode 100644 index 00000000000..96d116ec6a2 --- /dev/null +++ b/tests/integration/test_local_storage/test.py @@ -0,0 +1,153 @@ +import logging +import os +import random +import string + +import pytest + +from helpers.cluster import ClickHouseCluster +from azure.storage.blob import BlobServiceClient +from test_storage_azure_blob_storage.test import azure_query + +NODE_NAME = "node" + + +def generate_cluster_def(port): + path = os.path.join( + os.path.dirname(os.path.realpath(__file__)), + "./_gen/disk_storage_conf.xml", + ) + os.makedirs(os.path.dirname(path), exist_ok=True) + with open(path, "w") as f: + f.write( + f""" + + + + object_storage + azure_blob_storage + plain_rewritable + http://azurite1:{port}/devstoreaccount1 + cont + true + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + 100000 + 100000 + 10 + 10 + + + + + +
+ blob_storage_disk +
+
+
+
+
+
+""" + ) + return path + + +insert_values = [ + "(0,'data'),(1,'data')", + ",".join( + f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" + for i in range(10) + ), +] + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + port = cluster.azurite_port + path = generate_cluster_def(port) + cluster.add_instance( + NODE_NAME, + main_configs=[ + path, + ], + with_azurite=True, + stay_alive=True, + ) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + yield cluster + finally: + cluster.shutdown() + + +def test_insert_select(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + azure_query( + node, + """ + CREATE TABLE test_{} ( + id Int64, + data String + ) ENGINE=MergeTree() + ORDER BY id + SETTINGS storage_policy='blob_storage_policy' + """.format( + index + ), + ) + + azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_restart_server(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + node.restart_clickhouse() + + for index, value in enumerate(insert_values): + assert ( + azure_query( + node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) + ) + == value + ) + + +def test_drop_table(cluster): + node = cluster.instances[NODE_NAME] + + for index, value in enumerate(insert_values): + node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) + + port = cluster.env_variables["AZURITE_PORT"] + connection_string = ( + f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + ) + blob_service_client = BlobServiceClient.from_connection_string(connection_string) + containers = blob_service_client.list_containers() + for container in containers: + container_client = blob_service_client.get_container_client(container) + assert len(list(container_client.list_blobs())) == 0 From f54a4b073a3fd1c0d31b26c3085eedabb667687a Mon Sep 17 00:00:00 2001 From: divanik Date: Sat, 13 Jul 2024 14:07:36 +0000 Subject: [PATCH 02/33] Add local table function --- .../registerStorageObjectStorage.cpp | 24 +++++++++++++++++-- .../TableFunctionObjectStorage.h | 22 +++++++++++++---- 2 files changed, 40 insertions(+), 6 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index bf595b2f5d4..04c3417948c 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,9 +1,10 @@ +#include #include -#include #include +#include +#include #include #include -#include namespace DB { @@ -138,8 +139,27 @@ void registerStorageHDFS(StorageFactory & factory) } #endif +void registerStorageLocal(StorageFactory & factory) +{ + factory.registerStorage( + "Local", + [=](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::FILE, + }); +} + + void registerStorageObjectStorage(StorageFactory & factory) { + registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From d2eded16aaa62fe66e93bac171c25a1f031b3881 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 15 Jul 2024 10:53:22 +0000 Subject: [PATCH 03/33] Debug commit --- src/CMakeLists.txt | 1 + .../Local/LocalObjectStorage.cpp | 12 ++-- src/IO/ReadHelpers.cpp | 10 ++- .../Executors/PullingPipelineExecutor.cpp | 13 ++++ .../ObjectStorage/Local/Configuration.cpp | 2 + .../ObjectStorage/Local/Configuration.h | 20 +++--- .../ObjectStorage/StorageObjectStorage.cpp | 12 ++++ .../StorageObjectStorageSource.cpp | 62 ++++++++++++++++--- .../TableFunctionObjectStorage.cpp | 15 +++-- 9 files changed, 116 insertions(+), 31 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..dc32687e058 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -110,6 +110,7 @@ add_headers_and_sources(dbms Storages/ObjectStorage) add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) +add_headers_and_sources(dbms Storages/ObjectStorage/Local) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index a247d86ddce..6c21de96371 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,15 +1,17 @@ #include -#include -#include -#include +#include +#include +#include +#include #include #include -#include #include #include +#include +#include #include -#include +#include namespace fs = std::filesystem; diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index c771fced73a..ddc2e912fb3 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,6 +80,7 @@ UUID parseUUID(std::span src) return uuid; } + void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -88,12 +89,15 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); - throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); + throw Exception( + ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, + "Cannot parse input: expected {} {}", + out.str(), + buf.buffer().end() - buf.position()); } - bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..a4db27d3f37 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,19 +42,32 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { + LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); + if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } + LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); + if (!executor->checkTimeLimitSoft()) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); + + if (!executor->executeStep(&has_data_flag)) return false; + LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); + + chunk = pulling_format->getChunk(); + + LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); + return true; } diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 5b66dd2470b..3abdb4d530b 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -20,6 +20,7 @@ void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & coll format = collection.getOrDefault("format", "auto"); compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); + paths = {path}; } @@ -56,6 +57,7 @@ void StorageLocalConfiguration::fromAST(ASTs & args, ContextPtr context, bool wi { compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } + paths = {path}; } StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings(const ContextPtr & context) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index a305fc43825..11165fcd574 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -2,11 +2,13 @@ #include #include "Disks/ObjectStorages/Local/LocalObjectStorage.h" -#include "config.h" -#if USE_AWS_S3 -# include -# include +#include + +#include + + +namespace fs = std::filesystem; namespace DB { @@ -34,12 +36,12 @@ public: String getDataSourceDescription() const override { return ""; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; - void check(ContextPtr) const override { } - void validateNamespace(const String &) const override { } ConfigurationPtr clone() override { return std::make_shared(*this); } - bool isStaticConfiguration() const override { return true; } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared(path); } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override + { + return std::make_shared(fs::path{path}.parent_path()); + } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } @@ -51,5 +53,3 @@ private: }; } - -#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 683473006e3..8c5b4f71d4d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -124,6 +124,12 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { + LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } } std::string getName() const override { return name; } @@ -422,6 +428,12 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c DEFAULT_SCHEMA_CACHE_ELEMENTS)); return schema_cache; } + else if (storage_type_name == "local") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt("schema_inference_cache_max_elements_for_local", DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a9a7e062076..54ddaec0140 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,17 +1,18 @@ #include "StorageObjectStorageSource.h" -#include #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include +#include +#include +#include +#include +#include +#include +#include #include +#include +#include +#include "Common/logger_useful.h" #include namespace fs = std::filesystem; @@ -69,6 +70,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { + LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -131,6 +133,7 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) { auto keys = configuration->getPaths(); @@ -142,6 +145,19 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } + LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); + for (auto && key : configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); + } + + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); + for (auto && key : copy_configuration->getPaths()) + { + LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); + } + + iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -170,8 +186,11 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); + while (true) { + LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); + if (isCancelled() || !reader) { if (reader) @@ -179,10 +198,15 @@ Chunk StorageObjectStorageSource::generate() break; } + LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); + Chunk chunk; if (reader->pull(chunk)) { + LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); + UInt64 num_rows = chunk.getNumRows(); + LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -230,6 +254,9 @@ Chunk StorageObjectStorageSource::generate() return chunk; } + LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -300,6 +327,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); + LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); + QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -324,11 +353,17 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; + LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); + + std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; + LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); + + if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -341,6 +376,8 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { + LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); + CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -625,11 +662,18 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { + LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); + for (auto && key : keys) + { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); + } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? + for (auto && key : keys) { + LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..d156afb81a1 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,12 +12,13 @@ #include -#include #include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -203,6 +204,11 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif + factory.registerFunction>( + {.documentation + = {.description = R"(The table function can be used to read the data stored locally.)", + .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, + .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -223,4 +229,5 @@ template class TableFunctionObjectStorage; #endif +template class TableFunctionObjectStorage; } From 0bfe345a460766e8113934801609230410db1836 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 17 Jul 2024 09:24:33 +0000 Subject: [PATCH 04/33] Debug code --- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 7 ++++--- src/Storages/ObjectStorage/Local/Configuration.h | 5 +---- src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 7 +++++++ 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 6c21de96371..dfbc1a77d3c 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -50,10 +50,11 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { + LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); + auto from_file_storage = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index 11165fcd574..ba4de63ac47 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -38,10 +38,7 @@ public: ConfigurationPtr clone() override { return std::make_shared(*this); } - ObjectStoragePtr createObjectStorage(ContextPtr, bool) override - { - return std::make_shared(fs::path{path}.parent_path()); - } + ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared("/"); } void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 54ddaec0140..715f93c7f7e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -388,7 +388,12 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); + LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); + auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); + std::string answer(1000, ' '); + size_t read_bytes = new_read_buf->read(answer.data(), 1000); + LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -471,6 +476,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Read"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); From 84559ab31cac8d994fbccf9085884c1c2ea8f4b8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 10:06:06 +0000 Subject: [PATCH 05/33] Debug code --- .../Local/LocalObjectStorage.cpp | 100 +++++++++--------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 75 ++++++++----- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageSource.cpp | 36 ++++--- 4 files changed, 120 insertions(+), 93 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 3704e4bb672..607ce41dd0f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -58,27 +59,28 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL std::string answer(1000, ' '); size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase 00"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; + LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, @@ -86,56 +88,58 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL } case RemoteFSReadMethod::threadpool: { - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); + // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); auto impl = std::make_unique( std::move(read_buffer_creator), objects, "file:", modified_settings, global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - auto impl3 = std::make_unique( - std::move(read_buffer_creator), - objects, - "file:", - modified_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + // auto impl3 = std::make_unique( + // std::move(read_buffer_creator), + // objects, + // "file:", + // modified_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + + // auto async_reader = std::make_unique( + // std::move(impl3), + // reader2, + // read_settings, + // global_context->getAsyncReadCounters(), + // global_context->getFilesystemReadPrefetchesLog()); + + // answer = std::string(1000, ' '); + // read_bytes = async_reader->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - std::move(impl3), + return std::make_unique( + std::move(impl), reader, read_settings, global_context->getAsyncReadCounters(), global_context->getFilesystemReadPrefetchesLog()); - - answer = std::string(1000, ' '); - read_bytes = async_reader->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - "Read bytes: {}, string: {}", - read_bytes, - answer.substr(0, read_bytes)); - - // reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), reader, read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); } } } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 267c3eaaea4..351f167b6b9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,21 +200,41 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; + // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( + // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr + // { + // return std::make_unique( + // client.get(), + // uri.bucket, + // object_.remote_path, + // uri.version_id, + // settings_ptr->request_settings, + // disk_read_settings, + // /* use_external_buffer */ true, + // /* offset */ 0, + // /* read_until_position */ 0, + // restricted_seek); + // }; + + switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - auto impl2 = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ false); + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); return std::make_unique( std::move(read_buffer_creator), @@ -224,17 +244,24 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: - { + case RemoteFSReadMethod::threadpool: { + // auto impl2 = std::make_unique( + // std::move(read_buffer_creator2), + // objects, + // "s3:" + uri.bucket + "/", + // disk_read_settings, + // global_context->getFilesystemCacheLog(), + // /* use_external_buffer */ true); + + // std::string answer(1000, ' '); + // size_t read_bytes = impl2->read(answer.data(), 1000); + // LOG_DEBUG( + // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), + // "Read bytes: {}, string: {}", + // read_bytes, + // answer.substr(0, read_bytes)); + auto impl = std::make_unique( - std::move(read_buffer_creator), - objects, - "s3:" + uri.bucket + "/", - disk_read_settings, - global_context->getFilesystemCacheLog(), - /* use_external_buffer */true); - - auto impl2 = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", @@ -242,12 +269,6 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */ true); - std::string answer(1000, ' '); - size_t read_bytes = impl2->read(answer.data(), 1000); - LOG_DEBUG( - &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( std::move(impl), reader, disk_read_settings, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 818ce055c77..7a13ada224c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -157,7 +157,7 @@ public: ContextPtr local_context, bool with_table_structure); - /// Storage type: s3, hdfs, azure. + /// Storage type: s3, hdfs, azure, local. virtual std::string getTypeName() const = 0; /// Engine name: S3, HDFS, Azure. virtual std::string getEngineName() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index f664bed0204..8b800cbc343 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,6 +461,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); + read_settings.remote_fs_method = RemoteFSReadMethod::read; + LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); read_settings.enable_filesystem_cache = false; @@ -477,28 +479,28 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // 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 (use_prefetch) - { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + // if (use_prefetch) + // { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); - auto async_reader = object_storage->readObjects( - StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + auto async_reader + = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - return async_reader; - } - else - { - /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); + return async_reader; + // } + // else + // { + // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - } + // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + // } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) From 8fad2860251abf3fc71b525522c7ce51bd4687aa Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 18 Jul 2024 15:03:12 +0000 Subject: [PATCH 06/33] Remove asyncronous processing for local storage --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 5 +- .../Local/LocalObjectStorage.cpp | 96 ++----------------- .../ObjectStorage/Local/Configuration.cpp | 2 +- .../StorageObjectStorageSource.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 6 files changed, 20 insertions(+), 91 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 92b1c750a55..10b869e58f7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -114,6 +114,7 @@ class IColumn; M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ + M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 73fb28dc67a..5de4630758d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,8 +79,9 @@ static std::initializer_list LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr + auto read_buffer_creator = + [=] (bool /* restricted_seek */, const StoredObject & object) + -> std::unique_ptr { - LOG_DEBUG(&Poco::Logger::get("Read"), "Remote Path: {}", object.remote_path); - - auto reader = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - - std::string answer(1000, ' '); - size_t read_bytes = reader->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromFileBase"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - LOG_DEBUG(&Poco::Logger::get("Method"), "Method read: {}", read_settings.remote_fs_method == RemoteFSReadMethod::read); - switch (read_settings.remote_fs_method) - { - case RemoteFSReadMethod::read: - { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - return std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */false); - } - case RemoteFSReadMethod::threadpool: - { - // LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Threadpool"); - auto impl = std::make_unique( - std::move(read_buffer_creator), objects, "file:", modified_settings, - global_context->getFilesystemCacheLog(), /* use_external_buffer */true); - - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); - - // auto impl3 = std::make_unique( - // std::move(read_buffer_creator), - // objects, - // "file:", - // modified_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - - // auto & reader2 = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - // auto async_reader = std::make_unique( - // std::move(impl3), - // reader2, - // read_settings, - // global_context->getAsyncReadCounters(), - // global_context->getFilesystemReadPrefetchesLog()); - - // answer = std::string(1000, ' '); - // read_bytes = async_reader->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("AsynchronousBoundedReadBuffer"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - - auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - - return std::make_unique( - std::move(impl), - reader, - read_settings, - global_context->getAsyncReadCounters(), - global_context->getFilesystemReadPrefetchesLog()); - } - } + return std::make_unique( + std::move(read_buffer_creator), + objects, + "file:", + modified_settings, + global_context->getFilesystemCacheLog(), + /* use_external_buffer */ false); } ReadSettings LocalObjectStorage::patchSettings(const ReadSettings & read_settings) const diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 740132f0981..364bd21a64e 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -66,7 +66,7 @@ StorageObjectStorage::QuerySettings StorageLocalConfiguration::getQuerySettings( const auto & settings = context->getSettingsRef(); return StorageObjectStorage::QuerySettings{ .truncate_on_insert = settings.engine_file_truncate_on_insert, - .create_new_file_on_insert = false, + .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8b800cbc343..3053a48b75b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -461,7 +461,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - read_settings.remote_fs_method = RemoteFSReadMethod::read; + // read_settings.remote_fs_method = RemoteFSReadMethod::read; LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index e49e14d2a0c..6491deef440 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,7 +40,10 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); + configuration.getNamespace(), + key, + configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName()); } void resolveSchemaAndFormat( From 436f6463c03b5cfc56af30e49faa6e192750a8f5 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:06:30 +0000 Subject: [PATCH 07/33] Add test --- contrib/grpc | 2 +- .../test_local_storage/configs/config.xml | 3 + tests/integration/test_local_storage/test.py | 243 +++++++++--------- tests/integration/tmp_data/example.csv | 5 + 4 files changed, 131 insertions(+), 122 deletions(-) create mode 100644 tests/integration/test_local_storage/configs/config.xml create mode 100644 tests/integration/tmp_data/example.csv diff --git a/contrib/grpc b/contrib/grpc index f5b7fdc2dff..1716359d2e2 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit f5b7fdc2dff09ada06dbf6c75df298fb40f898df +Subproject commit 1716359d2e28d304a250f9df0e6c0ccad03de8db diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml new file mode 100644 index 00000000000..bcbb107f0a2 --- /dev/null +++ b/tests/integration/test_local_storage/configs/config.xml @@ -0,0 +1,3 @@ + + /var/lib/clickhouse/ + \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 96d116ec6a2..d27fb746d12 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -6,148 +6,149 @@ import string import pytest from helpers.cluster import ClickHouseCluster -from azure.storage.blob import BlobServiceClient -from test_storage_azure_blob_storage.test import azure_query -NODE_NAME = "node" - - -def generate_cluster_def(port): - path = os.path.join( - os.path.dirname(os.path.realpath(__file__)), - "./_gen/disk_storage_conf.xml", - ) - os.makedirs(os.path.dirname(path), exist_ok=True) - with open(path, "w") as f: - f.write( - f""" - - - - object_storage - azure_blob_storage - plain_rewritable - http://azurite1:{port}/devstoreaccount1 - cont - true - devstoreaccount1 - Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== - 100000 - 100000 - 10 - 10 - - - - - -
- blob_storage_disk -
-
-
-
-
-
-""" - ) - return path - - -insert_values = [ - "(0,'data'),(1,'data')", - ",".join( - f"({i},'{''.join(random.choices(string.ascii_lowercase, k=5))}')" - for i in range(10) - ), -] +from pathlib import Path @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): + global cluster try: cluster = ClickHouseCluster(__file__) - port = cluster.azurite_port - path = generate_cluster_def(port) cluster.add_instance( - NODE_NAME, - main_configs=[ - path, - ], - with_azurite=True, - stay_alive=True, + "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True ) - logging.info("Starting cluster...") - cluster.start() - logging.info("Cluster started") + cluster.start() yield cluster + finally: cluster.shutdown() -def test_insert_select(cluster): - node = cluster.instances[NODE_NAME] +def process_result(line: str): + return sorted( + list( + map( + lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), + filter(lambda x: len(x) > 0, line.split("\n")), + ) + ) + ) - for index, value in enumerate(insert_values): - azure_query( - node, - """ - CREATE TABLE test_{} ( + +def test_local_engine(started_cluster): + node = started_cluster.instances["test_local_storage"] + node.query( + """ + CREATE TABLE test_0 ( id Int64, data String - ) ENGINE=MergeTree() - ORDER BY id - SETTINGS storage_policy='blob_storage_policy' - """.format( - index - ), - ) + ) ENGINE=Local('/data/example.csv', 'CSV'); + """ + ) - azure_query(node, "INSERT INTO test_{} VALUES {}".format(index, value)) - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """ + ) + result = node.query( + """ + select * from test_0; + """ + ) -def test_restart_server(cluster): - node = cluster.instances[NODE_NAME] + assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) + + error_got = node.query_and_get_error( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """ + ) + + print("Error got", error_got) + + node.query( + """ + SET engine_file_truncate_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); + """, + settings={"engine_file_truncate_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + node.query( + """ + SET local_create_new_file_on_insert = 1; + """ + ) + + node.query( + """ + INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); + """, + settings={"local_create_new_file_on_insert": 1}, + ) + + result = node.query( + """ + SELECT * FROM test_0; + """ + ) + + assert [ + (-1, "7"), + (0, "15"), + (1, "3"), + (4, "abc"), + (5, "arr"), + (9, "ty"), + ] == process_result(result) - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) node.restart_clickhouse() - for index, value in enumerate(insert_values): - assert ( - azure_query( - node, "SELECT * FROM test_{} ORDER BY id FORMAT Values".format(index) - ) - == value - ) - - -def test_drop_table(cluster): - node = cluster.instances[NODE_NAME] - - for index, value in enumerate(insert_values): - node.query("DROP TABLE IF EXISTS test_{} SYNC".format(index)) - - port = cluster.env_variables["AZURITE_PORT"] - connection_string = ( - f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - f"BlobEndpoint=http://127.0.0.1:{port}/devstoreaccount1;" + result = node.query( + """ + SELECT * FROM test_0; + """ ) - blob_service_client = BlobServiceClient.from_connection_string(connection_string) - containers = blob_service_client.list_containers() - for container in containers: - container_client = blob_service_client.get_container_client(container) - assert len(list(container_client.list_blobs())) == 0 + + assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) + + +def test_table_function(started_cluster): + with open("/tmp/example.csv", "w") as f: + f.write( + """id,data +1,Str1 +2,Str2""" + ) + node = started_cluster.instances["test_local_storage"] + + node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + + result = node.query( + """ + SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); + """ + ) + + print("Res5", result) + + assert [(1, "Str1"), (2, "Str2")] == process_result(result) + + # assert False diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv new file mode 100644 index 00000000000..93d6fb20f38 --- /dev/null +++ b/tests/integration/tmp_data/example.csv @@ -0,0 +1,5 @@ + +id,data +1,'Str1' +2,'Str2' + \ No newline at end of file From 27ab6aa8b52118b3373aaa9ff3ccb8320a5f5344 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:16:24 +0000 Subject: [PATCH 08/33] Remove logs --- .../Local/LocalObjectStorage.cpp | 14 +-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 53 +-------- .../Executors/PullingPipelineExecutor.cpp | 22 +--- .../StorageObjectStorageSource.cpp | 107 ++++-------------- 4 files changed, 35 insertions(+), 161 deletions(-) diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index 266e39f8f97..01ff2e2baf1 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -1,18 +1,15 @@ #include -#include -#include -#include -#include +#include +#include +#include #include #include +#include #include #include -#include -#include -#include #include -#include +#include namespace fs = std::filesystem; @@ -58,7 +55,6 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); }; - return std::make_unique( std::move(read_buffer_creator), objects, diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 351f167b6b9..a6672e14e10 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -200,42 +200,10 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT restricted_seek); }; - // auto read_buffer_creator2 = [this, settings_ptr, disk_read_settings]( - // bool restricted_seek, const StoredObject & object_) -> std::unique_ptr - // { - // return std::make_unique( - // client.get(), - // uri.bucket, - // object_.remote_path, - // uri.version_id, - // settings_ptr->request_settings, - // disk_read_settings, - // /* use_external_buffer */ true, - // /* offset */ 0, - // /* read_until_position */ 0, - // restricted_seek); - // }; - - switch (read_settings.remote_fs_method) { case RemoteFSReadMethod::read: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ false); - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 000"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - return std::make_unique( std::move(read_buffer_creator), objects, @@ -244,30 +212,15 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT global_context->getFilesystemCacheLog(), /* use_external_buffer */false); } - case RemoteFSReadMethod::threadpool: { - // auto impl2 = std::make_unique( - // std::move(read_buffer_creator2), - // objects, - // "s3:" + uri.bucket + "/", - // disk_read_settings, - // global_context->getFilesystemCacheLog(), - // /* use_external_buffer */ true); - - // std::string answer(1000, ' '); - // size_t read_bytes = impl2->read(answer.data(), 1000); - // LOG_DEBUG( - // &Poco::Logger::get("ReadBufferFromRemoteFSGather 001"), - // "Read bytes: {}, string: {}", - // read_bytes, - // answer.substr(0, read_bytes)); - + case RemoteFSReadMethod::threadpool: + { auto impl = std::make_unique( std::move(read_buffer_creator), objects, "s3:" + uri.bucket + "/", disk_read_settings, global_context->getFilesystemCacheLog(), - /* use_external_buffer */ true); + /* use_external_buffer */true); auto & reader = global_context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); return std::make_unique( diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index a4db27d3f37..25c15d40c9a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -42,32 +42,19 @@ const Block & PullingPipelineExecutor::getHeader() const bool PullingPipelineExecutor::pull(Chunk & chunk) { - LOG_DEBUG(&Poco::Logger::get("Pulling 1"), "Pulling 1"); - if (!executor) { executor = std::make_shared(pipeline.processors, pipeline.process_list_element); executor->setReadProgressCallback(pipeline.getReadProgressCallback()); } - LOG_DEBUG(&Poco::Logger::get("Pulling 2"), "Pulling 2"); - if (!executor->checkTimeLimitSoft()) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 3"), "Pulling 3"); - - if (!executor->executeStep(&has_data_flag)) return false; - LOG_DEBUG(&Poco::Logger::get("Pulling 4"), "Pulling 4"); - - chunk = pulling_format->getChunk(); - - LOG_DEBUG(&Poco::Logger::get("Pulling 5"), "Pulling 5"); - return true; } @@ -86,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 3053a48b75b..086482e330a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -1,18 +1,17 @@ #include "StorageObjectStorageSource.h" +#include #include +#include +#include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include -#include "Common/logger_useful.h" +#include #include #include @@ -71,7 +70,6 @@ StorageObjectStorageSource::StorageObjectStorageSource( , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { - LOG_DEBUG(&Poco::Logger::get("Source created"), "Source created"); } StorageObjectStorageSource::~StorageObjectStorageSource() @@ -134,7 +132,6 @@ std::shared_ptr StorageObjectStorageSourc { ConfigurationPtr copy_configuration = configuration->clone(); auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_dag) { auto keys = configuration->getPaths(); @@ -146,19 +143,6 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } - - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Keys size: {}", copy_configuration->getPaths().size()); - for (auto && key : copy_configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Copy Conf"), "Current key: {}", key); - } - - iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); @@ -187,11 +171,8 @@ Chunk StorageObjectStorageSource::generate() { lazyInitialize(); - while (true) { - LOG_DEBUG(&Poco::Logger::get("Generating"), "Generating reader: {}", !(!reader)); - if (isCancelled() || !reader) { if (reader) @@ -199,15 +180,10 @@ Chunk StorageObjectStorageSource::generate() break; } - LOG_DEBUG(&Poco::Logger::get("Generating 2"), "Generating 2"); - Chunk chunk; if (reader->pull(chunk)) { - LOG_DEBUG(&Poco::Logger::get("Generating 3"), "Generating 3"); - UInt64 num_rows = chunk.getNumRows(); - LOG_DEBUG(&Poco::Logger::get("Creating_chunk"), "Chunk size: {}", num_rows); total_rows_in_file += num_rows; size_t chunk_size = 0; @@ -255,9 +231,6 @@ Chunk StorageObjectStorageSource::generate() return chunk; } - LOG_DEBUG(&Poco::Logger::get("Generating 4"), "Generating 4"); - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) addNumRowsToCache(*reader.getObjectInfo(), total_rows_in_file); @@ -328,8 +301,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); - LOG_DEBUG(&Poco::Logger::get("Unreached point 1"), ""); - QueryPipelineBuilder builder; std::shared_ptr source; std::unique_ptr read_buf; @@ -354,17 +325,11 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade return schema_cache->tryGetNumRows(cache_key, get_last_mod_time); }; - LOG_DEBUG(&Poco::Logger::get("Unreached point 2"), ""); - - std::optional num_rows_from_cache = need_only_count && context_->getSettingsRef().use_cache_for_count_from_files ? try_get_num_rows_from_cache() : std::nullopt; - LOG_DEBUG(&Poco::Logger::get("Unreached point 3"), ""); - - if (num_rows_from_cache) { /// We should not return single chunk with all number of rows, @@ -377,8 +342,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { - LOG_DEBUG(&Poco::Logger::get("Unreached point 4"), ""); - CompressionMethod compression_method; if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { @@ -389,12 +352,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); - LOG_DEBUG(&Poco::Logger::get("Info relative path"), "Info: {}", object_info->relative_path); read_buf = createReadBuffer(*object_info, object_storage, context_, log); - auto new_read_buf = createReadBuffer(*object_info, object_storage, context_, log); - std::string answer(1000, ' '); - size_t read_bytes = new_read_buf->read(answer.data(), 1000); - LOG_DEBUG(&Poco::Logger::get("Read buffer"), "Read bytes: {}, string: {}", read_bytes, answer.substr(0, read_bytes)); } auto input_format = FormatFactory::instance().getInput( @@ -460,17 +418,11 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( const auto & object_size = object_info.metadata->size_bytes; auto read_settings = context_->getReadSettings().adjustBufferSize(object_size); - - // read_settings.remote_fs_method = RemoteFSReadMethod::read; - - LOG_DEBUG(&Poco::Logger::get("Threadpool"), "Method threadpool: {}", read_settings.remote_fs_method == RemoteFSReadMethod::threadpool); - read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; const bool object_too_small = object_size <= 2 * context_->getSettingsRef().max_download_buffer_size; - const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; /// User's object may change, don't cache it. @@ -479,28 +431,24 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( // 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 (use_prefetch) - // { - LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + if (use_prefetch) + { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Read objects"), "Path: {}, object size: {}", object_info.getPath(), object_size); + auto async_reader = object_storage->readObjects( + StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); - auto async_reader - = object_storage->readObjects(StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; - // } - // else - // { - // /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - // LOG_DEBUG(&Poco::Logger::get("Read object"), "Path: {}, object size: {}", object_info.getPath(), object_size); - - // return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); - // } + return async_reader; + } + else + { + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); + } } StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) @@ -678,18 +626,11 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( , keys(configuration->getPaths()) , ignore_non_existent_files(ignore_non_existent_files_) { - LOG_DEBUG(&Poco::Logger::get("Keys size"), "Keys size: {}", keys.size()); - for (auto && key : keys) - { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); - } if (read_keys_) { /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? - for (auto && key : keys) { - LOG_DEBUG(&Poco::Logger::get("Current Key"), "Current key: {}", key); auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } From 45e0f0350e475503cdd2ac0a30d14ca2466fceb9 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:25:37 +0000 Subject: [PATCH 09/33] Fix some stuff --- .../integration/test_local_storage/files/example2.csv | 3 +++ tests/integration/test_local_storage/test.py | 10 +--------- tests/integration/tmp_data/example.csv | 5 ----- 3 files changed, 4 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/tmp_data/example.csv diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv new file mode 100644 index 00000000000..1abfb225937 --- /dev/null +++ b/tests/integration/test_local_storage/files/example2.csv @@ -0,0 +1,3 @@ +id,data +1,Str1 +2,Str2 \ No newline at end of file diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index d27fb746d12..54c8016c376 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -68,8 +68,6 @@ def test_local_engine(started_cluster): """ ) - print("Error got", error_got) - node.query( """ SET engine_file_truncate_on_insert = 1; @@ -131,15 +129,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): - with open("/tmp/example.csv", "w") as f: - f.write( - """id,data -1,Str1 -2,Str2""" - ) node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("/tmp/example.csv", "/data/example2.csv") + node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") result = node.query( """ diff --git a/tests/integration/tmp_data/example.csv b/tests/integration/tmp_data/example.csv deleted file mode 100644 index 93d6fb20f38..00000000000 --- a/tests/integration/tmp_data/example.csv +++ /dev/null @@ -1,5 +0,0 @@ - -id,data -1,'Str1' -2,'Str2' - \ No newline at end of file From 07d03c0c67437eaf70fe70199e71430d643302b6 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:30:46 +0000 Subject: [PATCH 10/33] Fix minor things --- src/IO/ReadHelpers.cpp | 10 +++------- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 6 ------ .../integration/test_local_storage/configs/config.xml | 2 +- .../integration/test_local_storage/files/example2.csv | 2 +- 4 files changed, 5 insertions(+), 15 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index ddc2e912fb3..c771fced73a 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -80,7 +80,6 @@ UUID parseUUID(std::span src) return uuid; } - void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; @@ -89,15 +88,12 @@ void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) if (buf.eof()) out << " at end of stream."; else - out << " before: " << quote << String(buf.position(), std::min(static_cast(1000), buf.buffer().end() - buf.position())); + out << " before: " << quote << String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())); - throw Exception( - ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, - "Cannot parse input: expected {} {}", - out.str(), - buf.buffer().end() - buf.position()); + throw Exception(ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED, "Cannot parse input: expected {}", out.str()); } + bool checkString(const char * s, ReadBuffer & buf) { for (; *s; ++s) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 0be929bc2c1..10fffd148be 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -125,12 +125,6 @@ public: , num_streams(num_streams_) , distributed_processing(distributed_processing_) { - LOG_DEBUG(&Poco::Logger::get("Read step created"), "Read step created"); - LOG_DEBUG(&Poco::Logger::get("Conf"), "Keys size: {}", configuration->getPaths().size()); - for (auto && key : configuration->getPaths()) - { - LOG_DEBUG(&Poco::Logger::get("Conf"), "Current key: {}", key); - } } std::string getName() const override { return name; } diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml index bcbb107f0a2..b4179ee51df 100644 --- a/tests/integration/test_local_storage/configs/config.xml +++ b/tests/integration/test_local_storage/configs/config.xml @@ -1,3 +1,3 @@ /var/lib/clickhouse/ - \ No newline at end of file + diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv index 1abfb225937..7b6e6d6bab1 100644 --- a/tests/integration/test_local_storage/files/example2.csv +++ b/tests/integration/test_local_storage/files/example2.csv @@ -1,3 +1,3 @@ id,data 1,Str1 -2,Str2 \ No newline at end of file +2,Str2 From f0ef101bc558400c1f38666a390862da71e1c0f2 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 13:40:30 +0000 Subject: [PATCH 11/33] Minor changes --- src/Storages/ObjectStorage/Local/Configuration.cpp | 1 - tests/integration/test_local_storage/test.py | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index 364bd21a64e..d64b22769ad 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int LOGICAL_ERROR; } void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & collection, ContextPtr) diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py index 54c8016c376..89c3c17e1f0 100644 --- a/tests/integration/test_local_storage/test.py +++ b/tests/integration/test_local_storage/test.py @@ -131,7 +131,9 @@ def test_local_engine(started_cluster): def test_table_function(started_cluster): node = started_cluster.instances["test_local_storage"] - node.copy_file_to_container("test_local_storage/files/example2.csv", "/data/example2.csv") + node.copy_file_to_container( + "test_local_storage/files/example2.csv", "/data/example2.csv" + ) result = node.query( """ From 57181a5a4871b251c3d1aa04a39d467a715645a4 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 19 Jul 2024 14:47:57 +0000 Subject: [PATCH 12/33] Fix compilation bug --- src/Storages/ObjectStorage/registerStorageObjectStorage.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 13f2df7a0e7..4bc8cfa6a2f 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -10,8 +10,6 @@ namespace DB { -#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS - namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -71,8 +69,6 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } -#endif - #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { From c59949d057bba311f040f7b6386b1f6b481a23dd Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 22 Jul 2024 09:50:47 +0000 Subject: [PATCH 13/33] Add different iceberg tables --- .../DataLakes/registerDataLakeStorages.cpp | 60 +++++++++++++++++-- src/TableFunctions/ITableFunctionDataLake.h | 27 ++++++++- .../registerDataLakeTableFunctions.cpp | 27 +++++---- 3 files changed, 96 insertions(+), 18 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index 0fa6402e892..e75ab3201b8 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -2,10 +2,12 @@ #if USE_AWS_S3 -#include -#include -#include -#include +# include +# include +# include +# include +# include +# include namespace DB @@ -22,6 +24,54 @@ void registerStorageIceberg(StorageFactory & factory) auto configuration = std::make_shared(); StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergS3", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); + + factory.registerStorage( + "IcebergAzure", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); + + factory.registerStorage( + "IcebergLocal", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); @@ -29,7 +79,7 @@ void registerStorageIceberg(StorageFactory & factory) { .supports_settings = false, .supports_schema_inference = true, - .source_access_type = AccessType::S3, + .source_access_type = AccessType::FILE, }); } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index fe6e5b3e593..db8287f97bf 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -76,6 +76,21 @@ struct TableFunctionIcebergName static constexpr auto name = "iceberg"; }; +struct TableFunctionIcebergS3Name +{ + static constexpr auto name = "icebergS3"; +}; + +struct TableFunctionIcebergAzureName +{ + static constexpr auto name = "icebergAzure"; +}; + +struct TableFunctionIcebergLocalName +{ + static constexpr auto name = "icebergLocal"; +}; + struct TableFunctionDeltaLakeName { static constexpr auto name = "deltaLake"; @@ -86,14 +101,20 @@ struct TableFunctionHudiName static constexpr auto name = "hudi"; }; -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 using TableFunctionIceberg = ITableFunctionDataLake; +using TableFunctionIcebergS3 = ITableFunctionDataLake; +# endif +# if USE_AZURE_BLOB_STORAGE +using TableFunctionIcebergAzure = ITableFunctionDataLake; +# endif +using TableFunctionIcebergLocal = ITableFunctionDataLake; #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET using TableFunctionDeltaLake = ITableFunctionDataLake; #endif using TableFunctionHudi = ITableFunctionDataLake; #endif - } diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 15a6668f434..40561a1e075 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -4,24 +4,31 @@ namespace DB { -#if USE_AWS_S3 #if USE_AVRO void registerTableFunctionIceberg(TableFunctionFactory & factory) { +# if USE_AWS_S3 factory.registerFunction( - { - .documentation = - { - .description=R"(The table function can be used to read the Iceberg table stored on object store.)", + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store. Alias to icebergS3)", .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"} - }, - .allow_readonly = false - }); + .categories{"DataLake"}}, + .allow_readonly = false}); + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); + +# endif +# if USE_AZURE_BLOB_STORAGE +# endif } #endif -#if USE_PARQUET +#if USE_AWS_S3 +# if USE_PARQUET void registerTableFunctionDeltaLake(TableFunctionFactory & factory) { factory.registerFunction( From 7e2e24c75d705922e16fc79a257d03d5d5c1017e Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 10:29:12 +0000 Subject: [PATCH 14/33] Unifiing tests changes --- .../test/integration/runner/requirements.txt | 1 + .../ObjectStorage/Azure/Configuration.cpp | 10 +- .../DataLakes/registerDataLakeStorages.cpp | 2 +- .../helpers/{s3_tools.py => cloud_tools.py} | 78 +++-- .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 + .../configs/users.d/users.xml | 9 + .../test_iceberg_azure_storage/test.py | 291 ++++++++++++++++++ tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 214 ++++++++++--- tests/integration/test_storage_s3/test.py | 2 +- 12 files changed, 552 insertions(+), 68 deletions(-) rename tests/integration/helpers/{s3_tools.py => cloud_tools.py} (53%) create mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py create mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml create mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml create mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8a77d8abf77..db10398e73d 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,6 +64,7 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 +pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index f0a0a562b92..9730391d429 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -148,10 +148,12 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, { if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage AzureBlobStorage requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " - "[account_name, account_key, format, compression, structure)])"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage AzureBlobStorage requires 3 to {} arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure)])", + (with_structure ? 8 : 7)); } for (auto & engine_arg : engine_args) diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index e75ab3201b8..f0bd51de375 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -54,7 +54,7 @@ void registerStorageIceberg(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), true); return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/cloud_tools.py similarity index 53% rename from tests/integration/helpers/s3_tools.py rename to tests/integration/helpers/cloud_tools.py index 0c3538c3c39..534791b8bc5 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -2,30 +2,66 @@ from minio import Minio import glob import os import json +import shutil -def upload_directory(minio_client, bucket_name, local_path, s3_path): - result_files = [] - for local_file in glob.glob(local_path + "/**"): - if os.path.isfile(local_file): +from enum import Enum + + +class CloudUploader: + def upload_directory(self, local_path, remote_blob_path): + result_files = [] + # print(f"Arguments: {local_path}, {s3_path}") + # for local_file in glob.glob(local_path + "/**"): + # print("Local file: {}", local_file) + for local_file in glob.glob(local_path + "/**"): result_local_path = os.path.join(local_path, local_file) - result_s3_path = os.path.join(s3_path, local_file) - print(f"Putting file {result_local_path} to {result_s3_path}") - minio_client.fput_object( - bucket_name=bucket_name, - object_name=result_s3_path, - file_path=result_local_path, - ) - result_files.append(result_s3_path) - else: - files = upload_directory( - minio_client, - bucket_name, - os.path.join(local_path, local_file), - os.path.join(s3_path, local_file), - ) - result_files.extend(files) - return result_files + result_remote_blob_path = os.path.join(remote_blob_path, local_file) + if os.path.isfile(local_file): + self.upload_file(result_local_path, result_remote_blob_path) + result_files.append(result_remote_blob_path) + else: + files = self.upload_directory( + result_local_path, + result_remote_blob_path, + ) + result_files.extend(files) + return result_files + + +class S3Uploader(CloudUploader): + def __init__(self, minio_client, bucket_name): + self.minio_client = minio_client + self.bucket_name = bucket_name + + def upload_file(self, local_path, remote_blob_path): + self.minio_client.fput_object( + bucket_name=self.bucket_name, + object_name=remote_blob_path, + file_path=local_path, + ) + + +class LocalUploader(CloudUploader): + def __init__(self): + pass + + def upload_file(self, local_path, remote_blob_path): + if local_path != remote_blob_path: + shutil.copyfile(local_path, remote_blob_path) + + +class AzureUploader(CloudUploader): + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_name = container_name + + def upload_file(self, local_path, remote_blob_path): + blob_client = self.blob_service_client.get_blob_client( + container=self.container_name, blob=remote_blob_path + ) + with open(local_path, "rb") as data: + blob_client.upload_blob(data, overwrite=True) def get_file_contents(minio_client, bucket, s3_path): diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml new file mode 100644 index 00000000000..d4c54e2d13d --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml @@ -0,0 +1,9 @@ + + + + http://minio1:9001/root/ + minio + minio123 + + + diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml new file mode 100644 index 00000000000..4b6ba057ecb --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml @@ -0,0 +1,9 @@ + + + + + default + 1 + + + diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py new file mode 100644 index 00000000000..f96f8acfaaf --- /dev/null +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -0,0 +1,291 @@ +import helpers.client +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.test_tools import TSV + +import pyspark +import logging +import os +import json +import pytest +import time +import glob +import uuid +import os + +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + +from pyspark.sql.types import ( + StructType, + StructField, + StringType, + IntegerType, + DateType, + TimestampType, + BooleanType, + ArrayType, +) +from pyspark.sql.functions import current_timestamp +from datetime import datetime +from pyspark.sql.functions import monotonically_increasing_id, row_number +from pyspark.sql.window import Window +from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 +from minio.deleteobjects import DeleteObject + +from tests.integration.helpers.cloud_tools import ( + prepare_s3_bucket, + upload_directory, + get_file_contents, + list_s3_objects, +) + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + + +def get_spark(): + builder = ( + pyspark.sql.SparkSession.builder.appName("spark_test") + .config( + "spark.sql.catalog.spark_catalog", + "org.apache.iceberg.spark.SparkSessionCatalog", + ) + .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") + .config( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", + ) + .master("local") + ) + return builder.master("local").getOrCreate() + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, with_spark=True) + cluster.add_instance( + "node1", + main_configs=["configs/config.d/named_collections.xml"], + user_configs=["configs/users.d/users.xml"], + with_minio=True, + stay_alive=True, + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + cluster.spark_session = get_spark() + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def write_iceberg_from_file( + spark, path, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( + partition_by + ).tableProperty("format-version", format_version).using("iceberg").create() + else: + spark.read.load(f"file://{path}").writeTo(table_name).append() + + +def write_iceberg_from_df( + spark, df, table_name, mode="overwrite", format_version="1", partition_by=None +): + if mode == "overwrite": + if partition_by is None: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).using("iceberg").create() + else: + df.writeTo(table_name).tableProperty( + "format-version", format_version + ).partitionedBy(partition_by).using("iceberg").create() + else: + df.writeTo(table_name).append() + + +def generate_data(spark, start, end): + a = spark.range(start, end, 1).toDF("a") + b = spark.range(start + 1, end + 1, 1).toDF("b") + b = b.withColumn("b", b["b"].cast(StringType())) + + a = a.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + b = b.withColumn( + "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) + ) + + df = a.join(b, on=["row_index"]).drop("row_index") + return df + + +def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + + +def create_initial_data_file( + cluster, node, query, table_name, compression_method="none" +): + node.query( + f""" + INSERT INTO TABLE FUNCTION + file('{table_name}.parquet') + SETTINGS + output_format_parquet_compression_method='{compression_method}', + s3_truncate_on_insert=1 {query} + FORMAT Parquet""" + ) + user_files_path = os.path.join( + SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" + ) + result_path = f"{user_files_path}/{table_name}.parquet" + return result_path + + +@pytest.mark.parametrize("format_version", ["1", "2"]) +def test_single_iceberg_file(started_cluster, format_version): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + minio_client = started_cluster.minio_client + bucket = started_cluster.minio_bucket + TABLE_NAME = "test_single_iceberg_file_" + format_version + + inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" + parquet_data_path = create_initial_data_file( + started_cluster, instance, inserted_data, TABLE_NAME + ) + + write_iceberg_from_file( + spark, parquet_data_path, TABLE_NAME, format_version=format_version + ) + + files = upload_directory( + minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + ) + + for bucket in minio_client.list_buckets(): + for object in minio_client.list_objects(bucket.name, recursive=True): + print("Object: ", object.object_name) + extension = object.object_name.split(".")[-1] + print("File extension: ", extension) + try: + response = minio_client.get_object( + object.bucket_name, object.object_name + ) + + if extension == "avro": + avro_bytes = response.read() + + # Use BytesIO to create a file-like object from the byte string + avro_file = io.BytesIO(avro_bytes) + + # Read the Avro data + reader = avro.datafile.DataFileReader( + avro_file, avro.io.DatumReader() + ) + records = [record for record in reader] + + # Close the reader + reader.close() + + # Now you can work with the records + for record in records: + # print(json.dumps(record, indent=4, sort_keys=True)) + print(str(record)) + # my_json = ( + # str(record) + # .replace("'", '"') + # .replace("None", "null") + # .replace('b"', '"') + # ) + # print(my_json) + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + elif extension == "json": + my_bytes_value = response.read() + my_json = my_bytes_value.decode("utf8").replace("'", '"') + data = json.loads(my_json) + s = json.dumps(data, indent=4, sort_keys=True) + print(s) + elif extension == "parquet": + # print("To be continued...") + # # Your byte string containing the Parquet data + # parquet_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # parquet_file = io.BytesIO(parquet_bytes) + + # # Read the Parquet data into a PyArrow Table + # table = pq.read_table(parquet_file) + + # # Convert the PyArrow Table to a Pandas DataFrame + # df = table.to_pandas() + + # # Now you can work with s DataFrame + # print(df) + parquet_bytes = ( + response.read() + ) # Replace with your actual byte string + + # Create a temporary file and write the byte string to it + with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + tmp_file.write(parquet_bytes) + tmp_file_path = tmp_file.name + + # Read the Parquet file using PySpark + df = spark.read.parquet(tmp_file_path) + + # Show the DataFrame + print(df.toPandas()) + else: + print(response.read()) + + finally: + print("----------------") + response.close() + response.release_conn() + + create_iceberg_table(instance, TABLE_NAME) + + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( + inserted_data + ) + + assert 0 == 1 diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index d3dd7cfe52a..aaff1414d8d 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -28,7 +28,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 0c3fbfb3cda..750b77b29f3 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7762d17b96f..7f83846bd89 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,6 +12,14 @@ import glob import uuid import os +import tempfile + +import io +import avro.schema +import avro.io +import avro.datafile +import pandas as pd + from pyspark.sql.types import ( StructType, StructField, @@ -29,11 +37,13 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from helpers.s3_tools import ( +from helpers.cloud_tools import ( prepare_s3_bucket, - upload_directory, get_file_contents, list_s3_objects, + S3Uploader, + AzureUploader, + LocalUploader, ) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -67,6 +77,7 @@ def started_cluster(): main_configs=["configs/config.d/named_collections.xml"], user_configs=["configs/users.d/users.xml"], with_minio=True, + with_azurite=True, stay_alive=True, ) @@ -77,6 +88,15 @@ def started_cluster(): logging.info("S3 bucket created") cluster.spark_session = get_spark() + cluster.default_s3_uploader = S3Uploader( + cluster.minio_client, cluster.minio_bucket + ) + + container_name = "my_container" + + cluster.default_azurite_uploader = AzureUploader( + cluster.blob_service_client, container_name + ) yield cluster @@ -142,13 +162,25 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) +def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): + if storage_type == "local": + pass + elif storage_type == "s3": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + elif storage_type == "azure": + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) def create_initial_data_file( @@ -170,32 +202,134 @@ def create_initial_data_file( return result_path +def default_upload_directory(started_cluster, storage_type, local_path, remote_path): + if storage_type == "local": + return LocalUploader().upload_directory(local_path, remote_path) + elif storage_type == "s3": + return started_cluster.default_s3_uploader.upload_directory( + local_path, remote_path + ) + elif storage_type == "azure": + return started_cluster.default_azure_uploader.upload_directory( + local_path, remote_path + ) + else: + raise Exception("Unknown iceberg storage type: {}", storage_type) + + @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3"]) +def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket TABLE_NAME = "test_single_iceberg_file_" + format_version - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME + write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) + + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) + # for bucket in minio_client.list_buckets(): + # for object in minio_client.list_objects(bucket.name, recursive=True): + # print("Object: ", object.object_name) + # extension = object.object_name.split(".")[-1] + # print("File extension: ", extension) + # try: + # response = minio_client.get_object( + # object.bucket_name, object.object_name + # ) - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) + # if extension == "avro": + # avro_bytes = response.read() + + # # Use BytesIO to create a file-like object from the byte string + # avro_file = io.BytesIO(avro_bytes) + + # # Read the Avro data + # reader = avro.datafile.DataFileReader( + # avro_file, avro.io.DatumReader() + # ) + # records = [record for record in reader] + + # # Close the reader + # reader.close() + + # # Now you can work with the records + # for record in records: + # # print(json.dumps(record, indent=4, sort_keys=True)) + # print(str(record)) + # # my_json = ( + # # str(record) + # # .replace("'", '"') + # # .replace("None", "null") + # # .replace('b"', '"') + # # ) + # # print(my_json) + # # data = json.loads(my_json) + # # s = json.dumps(data, indent=4, sort_keys=True) + # # print(s) + # elif extension == "json": + # my_bytes_value = response.read() + # my_json = my_bytes_value.decode("utf8").replace("'", '"') + # data = json.loads(my_json) + # s = json.dumps(data, indent=4, sort_keys=True) + # print(s) + # elif extension == "parquet": + # # print("To be continued...") + # # # Your byte string containing the Parquet data + # # parquet_bytes = response.read() + + # # # Use BytesIO to create a file-like object from the byte string + # # parquet_file = io.BytesIO(parquet_bytes) + + # # # Read the Parquet data into a PyArrow Table + # # table = pq.read_table(parquet_file) + + # # # Convert the PyArrow Table to a Pandas DataFrame + # # df = table.to_pandas() + + # # # Now you can work with s DataFrame + # # print(df) + # parquet_bytes = ( + # response.read() + # ) # Replace with your actual byte string + + # # Create a temporary file and write the byte string to it + # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: + # tmp_file.write(parquet_bytes) + # tmp_file_path = tmp_file.name + + # # Read the Parquet file using PySpark + # df = spark.read.parquet(tmp_file_path) + + # # Show the DataFrame + # print(df.toPandas()) + # else: + # print(response.read()) + + # finally: + # print("----------------") + # response.close() + # response.release_conn() create_iceberg_table(instance, TABLE_NAME) + + # print("Debug Print") + + # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) + + # print(instance.query("SELECT number FROM numbers(100)")) + assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data + "SELECT number, toString(number + 1) FROM numbers(100)" ) + # assert 0 == 1 + @pytest.mark.parametrize("format_version", ["1", "2"]) def test_partition_by(started_cluster, format_version): @@ -215,7 +349,7 @@ def test_partition_by(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -240,7 +374,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', @@ -260,7 +394,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" ) assert len(files) == 9 @@ -302,7 +436,9 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory(minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}", "") + upload_directory( + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + ) create_iceberg_table(instance, TABLE_NAME) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 @@ -345,7 +481,7 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -354,7 +490,7 @@ def test_delete_files(started_cluster, format_version): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -369,14 +505,14 @@ def test_delete_files(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @@ -399,7 +535,7 @@ def test_evolved_schema(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -410,7 +546,7 @@ def test_evolved_schema(started_cluster, format_version): spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -437,7 +573,7 @@ def test_row_based_deletes(started_cluster): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -446,7 +582,7 @@ def test_row_based_deletes(started_cluster): spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -472,7 +608,7 @@ def test_schema_inference(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, format) @@ -527,7 +663,7 @@ def test_metadata_file_selection(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -559,7 +695,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): ) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME) @@ -586,7 +722,7 @@ def test_restart_broken(started_cluster): write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) create_iceberg_table(instance, TABLE_NAME, bucket=bucket) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 @@ -614,7 +750,7 @@ def test_restart_broken(started_cluster): minio_client.make_bucket(bucket) files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" + S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 40cbf4b44a6..fea828dc3a5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.s3_tools import prepare_s3_bucket +from helpers.cloud_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From f5e993df2a3ed28505a6fdc22db16306c0308733 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 7 Aug 2024 16:46:33 +0000 Subject: [PATCH 15/33] Add tests --- .../DataLakes/IStorageDataLake.h | 49 +- .../StorageObjectStorageSource.cpp | 2 + .../registerDataLakeTableFunctions.cpp | 20 +- tests/integration/helpers/cloud_tools.py | 13 +- .../test_storage_azure_blob_storage/test.py | 2 + .../configs/config.d/named_collections.xml | 4 + .../integration/test_storage_iceberg/test.py | 424 +++++++++--------- 7 files changed, 291 insertions(+), 223 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..123a629f395 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,10 +48,20 @@ public: ConfigurationPtr configuration = base_configuration->clone(); + try { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - configuration->setPaths(metadata->getDataFiles()); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); + + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); + } + configuration->setPaths(data_files); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -80,9 +90,20 @@ public: const std::optional & format_settings_, ContextPtr local_context) { + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); + auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); + + auto schema_from_metadata = metadata->getTableSchema(); + auto data_files = metadata->getDataFiles(); + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); + } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -90,7 +111,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(metadata->getDataFiles()); + configuration->setPaths(data_files); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -99,14 +120,33 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); + if (current_metadata && *current_metadata == *new_metadata) return; + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); + + current_metadata = std::move(new_metadata); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); + + auto data_files = current_metadata->getDataFiles(); + + LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); + + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); + for (auto & data_file : data_files) + { + LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); + } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(current_metadata->getDataFiles()); + updated_configuration->setPaths(data_files); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -148,7 +188,10 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); + current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); + LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index e760098f10f..399e4a56ba8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -438,6 +438,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); + auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 40561a1e075..8de852f4fba 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -17,13 +17,25 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {.description = R"(The table function can be used to read the Iceberg table stored on S3 object store.)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .examples{{"icebergS3", "SELECT * FROM icebergS3(url, access_key_id, secret_access_key)", ""}}, .categories{"DataLake"}}, .allow_readonly = false}); # endif # if USE_AZURE_BLOB_STORAGE + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored on Azure object store.)", + .examples{{"icebergAzure", "SELECT * FROM icebergAzure(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); # endif + factory.registerFunction( + {.documentation + = {.description = R"(The table function can be used to read the Iceberg table stored locally.)", + .examples{{"icebergLocal", "SELECT * FROM icebergLocal(filename)", ""}}, + .categories{"DataLake"}}, + .allow_readonly = false}); } #endif @@ -62,11 +74,13 @@ void registerTableFunctionHudi(TableFunctionFactory & factory) void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); -#if USE_AWS_S3 #if USE_AVRO +# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#if USE_PARQUET +#endif +#if USE_AWS_S3 +# if USE_PARQUET registerTableFunctionDeltaLake(factory); #endif registerTableFunctionHudi(factory); diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 534791b8bc5..0d009b5f3fe 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -52,15 +52,16 @@ class LocalUploader(CloudUploader): class AzureUploader(CloudUploader): - def __init__(self, blob_service_client, container_name): - self.blob_service_client = blob_service_client - self.container_name = container_name + + def __init__(self, container_client): + self.container_client = container_client def upload_file(self, local_path, remote_blob_path): - blob_client = self.blob_service_client.get_blob_client( - container=self.container_name, blob=remote_blob_path - ) + # print("Local path", local_path) + # print("Remote blob path", remote_blob_path) + blob_client = self.container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: + # print("Data", data) blob_client.upload_blob(data, overwrite=True) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 6fbe7634642..c5599d20519 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,6 +550,7 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] + print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -559,6 +560,7 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' + assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index d4c54e2d13d..3d80fa2ecf9 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -5,5 +5,9 @@ minio minio123 + + devstoreaccount1 + Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 7f83846bd89..b8a67600785 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -36,6 +36,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject +from azure.storage.blob import BlobServiceClient from helpers.cloud_tools import ( prepare_s3_bucket, @@ -92,12 +93,27 @@ def started_cluster(): cluster.minio_client, cluster.minio_bucket ) - container_name = "my_container" + cluster.azure_container_name = "mycontainer" - cluster.default_azurite_uploader = AzureUploader( - cluster.blob_service_client, container_name + # connection_string = ( + # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" + # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" + # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" + # ) + # local_blob_service_client = BlobServiceClient.from_connection_string( + # cluster.env_variables["AZURITE_CONNECTION_STRING"] + # ) + + local_blob_service_client = cluster.blob_service_client + + container_client = local_blob_service_client.create_container( + cluster.azure_container_name ) + cluster.container_client = container_client + + cluster.default_azure_uploader = AzureUploader(container_client) + yield cluster finally: @@ -162,10 +178,24 @@ def generate_data(spark, start, end): return df -def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwargs): +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + table_function=False, + **kwargs, +): if storage_type == "local": pass elif storage_type == "s3": + if "bucket" in kwargs: + bucket = kwargs["bucket"] + else: + bucket = cluster.minio_bucket + if table_function: + return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( f""" DROP TABLE IF EXISTS {table_name}; @@ -173,11 +203,15 @@ def create_iceberg_table(storage_type, node, table_name, format="Parquet", **kwa ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" ) elif storage_type == "azure": + if table_function: + return f""" + icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) + """ node.query( f""" DROP TABLE IF EXISTS {table_name}; CREATE TABLE {table_name} - ENGINE=IcebergAzure(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) else: raise Exception("Unknown iceberg storage type: {}", storage_type) @@ -218,11 +252,11 @@ def default_upload_directory(started_cluster, storage_type, local_path, remote_p @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -233,96 +267,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "", ) - # for bucket in minio_client.list_buckets(): - # for object in minio_client.list_objects(bucket.name, recursive=True): - # print("Object: ", object.object_name) - # extension = object.object_name.split(".")[-1] - # print("File extension: ", extension) - # try: - # response = minio_client.get_object( - # object.bucket_name, object.object_name - # ) - - # if extension == "avro": - # avro_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # avro_file = io.BytesIO(avro_bytes) - - # # Read the Avro data - # reader = avro.datafile.DataFileReader( - # avro_file, avro.io.DatumReader() - # ) - # records = [record for record in reader] - - # # Close the reader - # reader.close() - - # # Now you can work with the records - # for record in records: - # # print(json.dumps(record, indent=4, sort_keys=True)) - # print(str(record)) - # # my_json = ( - # # str(record) - # # .replace("'", '"') - # # .replace("None", "null") - # # .replace('b"', '"') - # # ) - # # print(my_json) - # # data = json.loads(my_json) - # # s = json.dumps(data, indent=4, sort_keys=True) - # # print(s) - # elif extension == "json": - # my_bytes_value = response.read() - # my_json = my_bytes_value.decode("utf8").replace("'", '"') - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - # elif extension == "parquet": - # # print("To be continued...") - # # # Your byte string containing the Parquet data - # # parquet_bytes = response.read() - - # # # Use BytesIO to create a file-like object from the byte string - # # parquet_file = io.BytesIO(parquet_bytes) - - # # # Read the Parquet data into a PyArrow Table - # # table = pq.read_table(parquet_file) - - # # # Convert the PyArrow Table to a Pandas DataFrame - # # df = table.to_pandas() - - # # # Now you can work with s DataFrame - # # print(df) - # parquet_bytes = ( - # response.read() - # ) # Replace with your actual byte string - - # # Create a temporary file and write the byte string to it - # with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - # tmp_file.write(parquet_bytes) - # tmp_file_path = tmp_file.name - - # # Read the Parquet file using PySpark - # df = spark.read.parquet(tmp_file_path) - - # # Show the DataFrame - # print(df.toPandas()) - # else: - # print(response.read()) - - # finally: - # print("----------------") - # response.close() - # response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - # print("Debug Print") - - # print(instance.query(f"SELECT * FROM {TABLE_NAME}")) - - # print(instance.query("SELECT number FROM numbers(100)")) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( "SELECT number, toString(number + 1) FROM numbers(100)" @@ -332,12 +277,11 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_partition_by(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_partition_by_" + format_version + TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -348,22 +292,23 @@ def test_partition_by(started_cluster, format_version): partition_by="a", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" ) assert len(files) == 14 # 10 partitiions + 4 metadata files - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 10 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_multiple_iceberg_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -373,9 +318,13 @@ def test_multiple_iceberg_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) + # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/version-hint.text', # '/iceberg_data/default/test_multiple_iceberg_files/metadata/3127466b-299d-48ca-a367-6b9b1df1e78c-m0.avro', @@ -383,7 +332,7 @@ def test_multiple_iceberg_files(started_cluster, format_version): # '/iceberg_data/default/test_multiple_iceberg_files/metadata/v1.metadata.json'] assert len(files) == 5 - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 write_iceberg_from_df( @@ -393,8 +342,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): mode="append", format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert len(files) == 9 @@ -405,12 +357,11 @@ def test_multiple_iceberg_files(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_types(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_types_" + format_version + TABLE_NAME = "test_types_" + format_version + "_" + storage_type data = [ ( @@ -436,24 +387,29 @@ def test_types(started_cluster, format_version): spark, df, TABLE_NAME, mode="overwrite", format_version=format_version ) - upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}", "" + default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 1 assert ( instance.query(f"SELECT a, b, c, d, e FROM {TABLE_NAME}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function = f"iceberg(s3, filename='iceberg_data/default/{TABLE_NAME}/')" + table_function_expr = create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, table_function=True + ) assert ( - instance.query(f"SELECT a, b, c, d, e FROM {table_function}").strip() + instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - assert instance.query(f"DESCRIBE {table_function} FORMAT TSV") == TSV( + assert instance.query(f"DESCRIBE {table_function_expr} FORMAT TSV") == TSV( [ ["a", "Nullable(Int32)"], ["b", "Nullable(String)"], @@ -465,12 +421,13 @@ def test_types(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_delete_files(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -480,17 +437,22 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 0") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 0 @@ -504,27 +466,34 @@ def test_delete_files(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE a >= 150") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 50 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_evolved_schema(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type write_iceberg_from_df( spark, @@ -534,19 +503,25 @@ def test_evolved_schema(started_cluster, format_version): format_version=format_version, ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 expected_data = instance.query(f"SELECT * FROM {TABLE_NAME} order by a, b") spark.sql(f"ALTER TABLE {TABLE_NAME} ADD COLUMNS (x bigint)") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -558,12 +533,13 @@ def test_evolved_schema(started_cluster, format_version): assert data == expected_data -def test_row_based_deletes(started_cluster): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes" + TABLE_NAME = "test_row_based_deletes_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -572,17 +548,23 @@ def test_row_based_deletes(started_cluster): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(100)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 spark.sql(f"DELETE FROM {TABLE_NAME} WHERE id < 10") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) error = instance.query_and_get_error(f"SELECT * FROM {TABLE_NAME}") @@ -590,13 +572,19 @@ def test_row_based_deletes(started_cluster): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_schema_inference(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket for format in ["Parquet", "ORC", "Avro"]: - TABLE_NAME = "test_schema_inference_" + format + "_" + format_version + TABLE_NAME = ( + "test_schema_inference_" + + format + + "_" + + format_version + + "_" + + storage_type + ) # Types time, timestamptz, fixed are not supported in Spark. spark.sql( @@ -606,12 +594,16 @@ def test_schema_inference(started_cluster, format_version): spark.sql( f"insert into {TABLE_NAME} select 42, 4242, 42.42, 4242.4242, decimal(42.42), decimal(42.42), decimal(42.42), date('2020-01-01'), timestamp('2020-01-01 20:00:00'), 'hello', binary('hello'), array(1,2,3), map('key', 'value'), struct(42, 'hello'), array(struct(map('key', array(map('key', 42))), struct(42, 'hello')))" ) - - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME, format) + create_iceberg_table( + storage_type, instance, TABLE_NAME, started_cluster, format=format + ) res = instance.query( f"DESC {TABLE_NAME} FORMAT TSVRaw", settings={"print_pretty_type_names": 0} @@ -646,12 +638,11 @@ def test_schema_inference(started_cluster, format_version): @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_selection(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_" + format_version + TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -662,22 +653,26 @@ def test_metadata_file_selection(started_cluster, format_version): f"INSERT INTO {TABLE_NAME} select id, char(id + ascii('a')) from range(10)" ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 @pytest.mark.parametrize("format_version", ["1", "2"]) -def test_metadata_file_format_with_uuid(started_cluster, format_version): +@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_metadata_selection_with_uuid_" + format_version + TABLE_NAME = ( + "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -694,63 +689,70 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version): f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", ) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" + files = default_upload_directory( + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + "", ) - create_iceberg_table(instance, TABLE_NAME) + create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -def test_restart_broken(started_cluster): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = "broken2" - TABLE_NAME = "test_restart_broken_table_function" +# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) +# def test_restart_broken(started_cluster): +# instance = started_cluster.instances["node1"] +# spark = started_cluster.spark_session +# minio_client = started_cluster.minio_client +# bucket = "broken2" +# TABLE_NAME = "test_restart_broken_table_function" - if not minio_client.bucket_exists(bucket): - minio_client.make_bucket(bucket) +# if not minio_client.bucket_exists(bucket): +# minio_client.make_bucket(bucket) - parquet_data_path = create_initial_data_file( - started_cluster, - instance, - "SELECT number, toString(number) FROM numbers(100)", - TABLE_NAME, - ) +# parquet_data_path = create_initial_data_file( +# started_cluster, +# instance, +# "SELECT number, toString(number) FROM numbers(100)", +# TABLE_NAME, +# ) - write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - create_iceberg_table(instance, TABLE_NAME, bucket=bucket) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") +# files = default_upload_directory( +# started_cluster, +# storage_type, +# f"/iceberg_data/default/{TABLE_NAME}/", +# "", +# ) +# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 - s3_objects = list_s3_objects(minio_client, bucket, prefix="") - assert ( - len( - list( - minio_client.remove_objects( - bucket, - [DeleteObject(obj) for obj in s3_objects], - ) - ) - ) - == 0 - ) - minio_client.remove_bucket(bucket) +# s3_objects = list_s3_objects(minio_client, bucket, prefix="") +# assert ( +# len( +# list( +# minio_client.remove_objects( +# bucket, +# [DeleteObject(obj) for obj in s3_objects], +# ) +# ) +# ) +# == 0 +# ) +# minio_client.remove_bucket(bucket) - instance.restart_clickhouse() +# instance.restart_clickhouse() - assert "NoSuchBucket" in instance.query_and_get_error( - f"SELECT count() FROM {TABLE_NAME}" - ) +# assert "NoSuchBucket" in instance.query_and_get_error( +# f"SELECT count() FROM {TABLE_NAME}" +# ) - minio_client.make_bucket(bucket) +# minio_client.make_bucket(bucket) - files = upload_directory( - S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" - ) +# files = default_upload_directory( +# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" +# ) - assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 +# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 29ce915d00f6a48a7dfa6a70ec9889d47eacf584 Mon Sep 17 00:00:00 2001 From: divanik Date: Fri, 9 Aug 2024 09:37:16 +0000 Subject: [PATCH 16/33] Try to fix bug --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 2 ++ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 ++ src/Disks/IO/createReadBufferFromFileBase.cpp | 28 +++++++++++++------ .../Local/LocalObjectStorage.cpp | 9 +++--- src/IO/ReadBuffer.h | 3 ++ .../StorageObjectStorageSource.cpp | 5 +--- 6 files changed, 33 insertions(+), 16 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index bb9761a3905..80dcc1baa14 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,6 +169,7 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { + // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); @@ -230,6 +231,7 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; + // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 9f1cb681f1a..a8d5bd1797e 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,6 +85,8 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; + std::deque buffer_cemetery_; + LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index b132e25ac6b..c6152543a41 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,14 +1,15 @@ +#include +#include #include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#include -#include -#include #include +#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -77,6 +78,7 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -88,6 +90,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); + res = std::make_unique( filename, buffer_size, @@ -99,6 +103,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); + #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -117,6 +123,8 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); + auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -131,6 +139,7 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -144,8 +153,11 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + { + LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); + } return res; }; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index c37b47bbc93..d291dcd65cf 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -48,11 +48,12 @@ std::unique_ptr LocalObjectStorage::readObjects( /// NOL { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); - auto read_buffer_creator = - [=] (bool /* restricted_seek */, const StoredObject & object) - -> std::unique_ptr + auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr { - return createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); + auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); + LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); + return kek; }; return std::make_unique( diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 73f5335411f..ea41aab84f0 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -103,6 +103,7 @@ public: */ bool ALWAYS_INLINE eof() { + LOG_DEBUG(); return !hasPendingData() && !next(); } @@ -182,6 +183,8 @@ public: while (bytes_copied < n && !eof()) { + auto k = *pos; + LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 399e4a56ba8..9233986d858 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -413,10 +413,7 @@ std::future StorageObjectStorageSource } std::unique_ptr StorageObjectStorageSource::createReadBuffer( - const ObjectInfo & object_info, - const ObjectStoragePtr & object_storage, - const ContextPtr & context_, - const LoggerPtr & log) + const ObjectInfo & object_info, const ObjectStoragePtr & object_storage, const ContextPtr & context_, const LoggerPtr & log) { const auto & object_size = object_info.metadata->size_bytes; From bc5d7933824d535bfbf2c672e5d7405c8bde86c8 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 09:13:24 +0000 Subject: [PATCH 17/33] Replace asyncronouos buffer with syncronouos --- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/createReadBufferFromFileBase.cpp | 9 --------- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 11 +++-------- src/IO/ReadBuffer.h | 8 +------- 4 files changed, 4 insertions(+), 25 deletions(-) diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 80dcc1baa14..01eb3d86696 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -231,7 +231,6 @@ void ReadBufferFromRemoteFSGather::reset() { current_object = StoredObject(); current_buf_idx = {}; - // buffer_cemetery_.push_back(current_buf); current_buf.reset(); } diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index c6152543a41..76588bcb5b1 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -78,7 +78,6 @@ std::unique_ptr createReadBufferFromFileBase( if (settings.local_fs_method == LocalFSReadMethod::read) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 1"); res = std::make_unique( filename, buffer_size, @@ -90,8 +89,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread || settings.local_fs_method == LocalFSReadMethod::mmap) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 2"); - res = std::make_unique( filename, buffer_size, @@ -103,7 +100,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 3"); #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); @@ -123,8 +119,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_fake_async) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 4"); - auto & reader = getThreadPoolReader(FilesystemReaderType::SYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -139,7 +133,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::pread_threadpool) { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 5"); auto & reader = getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_LOCAL_FS_READER); res = std::make_unique( reader, @@ -154,8 +147,6 @@ std::unique_ptr createReadBufferFromFileBase( } else { - LOG_DEBUG(&Poco::Logger::get("Read settings"), "Read settings 6"); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); } return res; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index d291dcd65cf..3b650adb71f 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -43,18 +43,13 @@ bool LocalObjectStorage::exists(const StoredObject & object) const std::unique_ptr LocalObjectStorage::readObjects( /// NOLINT const StoredObjects & objects, const ReadSettings & read_settings, - std::optional read_hint, - std::optional file_size) const + std::optional, + std::optional) const { auto modified_settings = patchSettings(read_settings); auto global_context = Context::getGlobalContextInstance(); auto read_buffer_creator = [=](bool /* restricted_seek */, const StoredObject & object) -> std::unique_ptr - { - LOG_DEBUG(&Poco::Logger::get("Get object path"), "Remote Path: {}", object.remote_path); - auto kek = createReadBufferFromFileBase(object.remote_path, modified_settings, read_hint, file_size); - LOG_DEBUG(&Poco::Logger::get("Buffer created"), "Remote Path: {}", object.remote_path); - return kek; - }; + { return std::make_unique(object.remote_path); }; return std::make_unique( std::move(read_buffer_creator), diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index ea41aab84f0..98f0c8c9059 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,11 +101,7 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() - { - LOG_DEBUG(); - return !hasPendingData() && !next(); - } + bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } void ignore() { @@ -183,8 +179,6 @@ public: while (bytes_copied < n && !eof()) { - auto k = *pos; - LOG_DEBUG(&Poco::Logger::get("Next symbol in read"), "Symbol: {}", k); size_t bytes_to_copy = std::min(static_cast(working_buffer.end() - pos), n - bytes_copied); ::memcpy(to + bytes_copied, pos, bytes_to_copy); pos += bytes_to_copy; From bbf10088869b0490fb81957b3417c7776f2c3089 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:15:54 +0000 Subject: [PATCH 18/33] Fix test --- tests/integration/helpers/cloud_tools.py | 59 ++++-- tests/integration/helpers/cluster.py | 1 + .../configs/config.d/named_collections.xml | 2 + .../integration/test_storage_iceberg/test.py | 194 ++++++++++-------- 4 files changed, 150 insertions(+), 106 deletions(-) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/cloud_tools.py index 0d009b5f3fe..39c56e0eb85 100644 --- a/tests/integration/helpers/cloud_tools.py +++ b/tests/integration/helpers/cloud_tools.py @@ -9,7 +9,9 @@ from enum import Enum class CloudUploader: - def upload_directory(self, local_path, remote_blob_path): + + def upload_directory(self, local_path, remote_blob_path, **kwargs): + print(kwargs) result_files = [] # print(f"Arguments: {local_path}, {s3_path}") # for local_file in glob.glob(local_path + "/**"): @@ -18,12 +20,11 @@ class CloudUploader: result_local_path = os.path.join(local_path, local_file) result_remote_blob_path = os.path.join(remote_blob_path, local_file) if os.path.isfile(local_file): - self.upload_file(result_local_path, result_remote_blob_path) + self.upload_file(result_local_path, result_remote_blob_path, **kwargs) result_files.append(result_remote_blob_path) else: files = self.upload_directory( - result_local_path, - result_remote_blob_path, + result_local_path, result_remote_blob_path, **kwargs ) result_files.extend(files) return result_files @@ -34,37 +35,61 @@ class S3Uploader(CloudUploader): self.minio_client = minio_client self.bucket_name = bucket_name - def upload_file(self, local_path, remote_blob_path): + def upload_file(self, local_path, remote_blob_path, bucket=None): + print(f"Upload to bucket: {bucket}") + if bucket is None: + bucket = self.bucket_name self.minio_client.fput_object( - bucket_name=self.bucket_name, + bucket_name=bucket, object_name=remote_blob_path, file_path=local_path, ) class LocalUploader(CloudUploader): - def __init__(self): - pass + + def __init__(self, clickhouse_node): + self.clickhouse_node = clickhouse_node def upload_file(self, local_path, remote_blob_path): - if local_path != remote_blob_path: - shutil.copyfile(local_path, remote_blob_path) + dir_path = os.path.dirname(remote_blob_path) + if dir_path != "": + self.clickhouse_node.exec_in_container( + [ + "bash", + "-c", + "mkdir -p {}".format(dir_path), + ] + ) + self.clickhouse_node.copy_file_to_container(local_path, remote_blob_path) class AzureUploader(CloudUploader): - def __init__(self, container_client): - self.container_client = container_client + def __init__(self, blob_service_client, container_name): + self.blob_service_client = blob_service_client + self.container_client = self.blob_service_client.get_container_client( + container_name + ) - def upload_file(self, local_path, remote_blob_path): - # print("Local path", local_path) - # print("Remote blob path", remote_blob_path) - blob_client = self.container_client.get_blob_client(remote_blob_path) + def upload_file(self, local_path, remote_blob_path, container_name=None): + if container_name is None: + container_client = self.container_client + else: + container_client = self.blob_service_client.get_container_client( + container_name + ) + blob_client = container_client.get_blob_client(remote_blob_path) with open(local_path, "rb") as data: - # print("Data", data) blob_client.upload_blob(data, overwrite=True) +def upload_directory(minio_client, bucket, local_path, remote_path): + S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + local_path, remote_path + ) + + def get_file_contents(minio_client, bucket, s3_path): data = minio_client.get_object(bucket, s3_path) data_str = b"" diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 215718463e8..7e85bcf2f3c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,6 +2204,7 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") + self.exec_in_container( container_id, [ diff --git a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml index 3d80fa2ecf9..b488638dd19 100644 --- a/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml +++ b/tests/integration/test_storage_iceberg/configs/config.d/named_collections.xml @@ -9,5 +9,7 @@ devstoreaccount1 Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw== + + diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b8a67600785..b1d9b7f66bf 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -95,24 +95,19 @@ def started_cluster(): cluster.azure_container_name = "mycontainer" - # connection_string = ( - # f"DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;" - # f"AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;" - # f"BlobEndpoint=http://azurite1:{cluster.env_variables['AZURITE_PORT']}/devstoreaccount1;" - # ) - # local_blob_service_client = BlobServiceClient.from_connection_string( - # cluster.env_variables["AZURITE_CONNECTION_STRING"] - # ) + cluster.blob_service_client = cluster.blob_service_client - local_blob_service_client = cluster.blob_service_client - - container_client = local_blob_service_client.create_container( + container_client = cluster.blob_service_client.create_container( cluster.azure_container_name ) cluster.container_client = container_client - cluster.default_azure_uploader = AzureUploader(container_client) + cluster.default_azure_uploader = AzureUploader( + cluster.blob_service_client, cluster.azure_container_name + ) + + cluster.default_local_uploader = LocalUploader(cluster.instances["node1"]) yield cluster @@ -187,13 +182,12 @@ def create_iceberg_table( table_function=False, **kwargs, ): - if storage_type == "local": - pass - elif storage_type == "s3": + if storage_type == "s3": if "bucket" in kwargs: bucket = kwargs["bucket"] else: bucket = cluster.minio_bucket + print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" node.query( @@ -213,8 +207,19 @@ def create_iceberg_table( CREATE TABLE {table_name} ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" ) + elif storage_type == "local": + if table_function: + return f""" + icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) + """ + node.query( + f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" + ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") def create_initial_data_file( @@ -236,23 +241,28 @@ def create_initial_data_file( return result_path -def default_upload_directory(started_cluster, storage_type, local_path, remote_path): +def default_upload_directory( + started_cluster, storage_type, local_path, remote_path, **kwargs +): if storage_type == "local": - return LocalUploader().upload_directory(local_path, remote_path) + return started_cluster.default_local_uploader.upload_directory( + local_path, remote_path, **kwargs + ) elif storage_type == "s3": + print(kwargs) return started_cluster.default_s3_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) elif storage_type == "azure": return started_cluster.default_azure_uploader.upload_directory( - local_path, remote_path + local_path, remote_path, **kwargs ) else: - raise Exception("Unknown iceberg storage type: {}", storage_type) + raise Exception(f"Unknown iceberg storage type: {storage_type}") @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -264,7 +274,7 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -273,11 +283,9 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): "SELECT number, toString(number + 1) FROM numbers(100)" ) - # assert 0 == 1 - @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -293,7 +301,10 @@ def test_partition_by(started_cluster, format_version, storage_type): ) files = default_upload_directory( - started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", "" + started_cluster, + storage_type, + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", ) assert len(files) == 14 # 10 partitiions + 4 metadata files @@ -302,7 +313,7 @@ def test_partition_by(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_multiple_iceberg_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -322,7 +333,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) # ['/iceberg_data/default/test_multiple_iceberg_files/data/00000-1-35302d56-f1ed-494e-a85b-fbf85c05ab39-00001.parquet', @@ -357,7 +368,7 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -391,7 +402,7 @@ def test_types(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -421,7 +432,7 @@ def test_types(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_delete_files(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -441,7 +452,7 @@ def test_delete_files(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -487,7 +498,7 @@ def test_delete_files(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_evolved_schema(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -507,7 +518,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -533,7 +544,7 @@ def test_evolved_schema(started_cluster, format_version, storage_type): assert data == expected_data -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_row_based_deletes(started_cluster, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -552,7 +563,7 @@ def test_row_based_deletes(started_cluster, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -572,7 +583,7 @@ def test_row_based_deletes(started_cluster, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_schema_inference(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -598,7 +609,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table( @@ -638,7 +649,7 @@ def test_schema_inference(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -657,7 +668,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -666,7 +677,7 @@ def test_metadata_file_selection(started_cluster, format_version, storage_type): @pytest.mark.parametrize("format_version", ["1", "2"]) -@pytest.mark.parametrize("storage_type", ["s3", "azure"]) +@pytest.mark.parametrize("storage_type", ["s3", "azure", "local"]) def test_metadata_file_format_with_uuid(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session @@ -693,7 +704,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage started_cluster, storage_type, f"/iceberg_data/default/{TABLE_NAME}/", - "", + f"/iceberg_data/default/{TABLE_NAME}/", ) create_iceberg_table(storage_type, instance, TABLE_NAME, started_cluster) @@ -701,58 +712,63 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 500 -# @pytest.mark.parametrize("storage_type", ["s3", "azure"]) -# def test_restart_broken(started_cluster): -# instance = started_cluster.instances["node1"] -# spark = started_cluster.spark_session -# minio_client = started_cluster.minio_client -# bucket = "broken2" -# TABLE_NAME = "test_restart_broken_table_function" +def test_restart_broken_s3(started_cluster): + instance = started_cluster.instances["node1"] + spark = started_cluster.spark_session + TABLE_NAME = "test_restart_broken_table_function_s3" -# if not minio_client.bucket_exists(bucket): -# minio_client.make_bucket(bucket) + minio_client = started_cluster.minio_client + bucket = "broken2" -# parquet_data_path = create_initial_data_file( -# started_cluster, -# instance, -# "SELECT number, toString(number) FROM numbers(100)", -# TABLE_NAME, -# ) + if not minio_client.bucket_exists(bucket): + minio_client.make_bucket(bucket) -# write_iceberg_from_file(spark, parquet_data_path, TABLE_NAME, format_version="1") -# files = default_upload_directory( -# started_cluster, -# storage_type, -# f"/iceberg_data/default/{TABLE_NAME}/", -# "", -# ) -# create_iceberg_table(instance, TABLE_NAME, bucket=bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + write_iceberg_from_df( + spark, + generate_data(spark, 0, 100), + TABLE_NAME, + mode="overwrite", + format_version="1", + ) -# s3_objects = list_s3_objects(minio_client, bucket, prefix="") -# assert ( -# len( -# list( -# minio_client.remove_objects( -# bucket, -# [DeleteObject(obj) for obj in s3_objects], -# ) -# ) -# ) -# == 0 -# ) -# minio_client.remove_bucket(bucket) + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + create_iceberg_table("s3", instance, TABLE_NAME, started_cluster, bucket=bucket) + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 -# instance.restart_clickhouse() + s3_objects = list_s3_objects(minio_client, bucket, prefix="") + assert ( + len( + list( + minio_client.remove_objects( + bucket, + [DeleteObject(obj) for obj in s3_objects], + ) + ) + ) + == 0 + ) + minio_client.remove_bucket(bucket) -# assert "NoSuchBucket" in instance.query_and_get_error( -# f"SELECT count() FROM {TABLE_NAME}" -# ) + instance.restart_clickhouse() -# minio_client.make_bucket(bucket) + assert "NoSuchBucket" in instance.query_and_get_error( + f"SELECT count() FROM {TABLE_NAME}" + ) -# files = default_upload_directory( -# S3Uploader(minio_client, bucket), f"/iceberg_data/default/{TABLE_NAME}/", "" -# ) + minio_client.make_bucket(bucket) -# assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 + files = default_upload_directory( + started_cluster, + "s3", + f"/iceberg_data/default/{TABLE_NAME}/", + f"/iceberg_data/default/{TABLE_NAME}/", + bucket=bucket, + ) + + assert int(instance.query(f"SELECT count() FROM {TABLE_NAME}")) == 100 From 0810703d6b162135a6f076e96052c433ef5b25d7 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:38:38 +0000 Subject: [PATCH 19/33] Roll out strange changes --- .../test/integration/runner/requirements.txt | 1 - src/Core/SettingsChangesHistory.cpp | 271 +----------------- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 +- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 1 - src/Disks/IO/ReadBufferFromRemoteFSGather.h | 2 - src/Disks/IO/createReadBufferFromFileBase.cpp | 19 +- src/IO/ReadBuffer.h | 5 +- .../DataLakes/IStorageDataLake.h | 49 +--- .../helpers/{cloud_tools.py => s3_tools.py} | 0 .../test_iceberg_azure_storage/test.py | 2 +- .../test_local_storage/__init__.py | 0 .../test_local_storage/configs/config.xml | 3 - .../test_local_storage/files/example2.csv | 3 - tests/integration/test_local_storage/test.py | 148 ---------- tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- .../integration/test_storage_iceberg/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 18 files changed, 26 insertions(+), 503 deletions(-) rename tests/integration/helpers/{cloud_tools.py => s3_tools.py} (100%) delete mode 100644 tests/integration/test_local_storage/__init__.py delete mode 100644 tests/integration/test_local_storage/configs/config.xml delete mode 100644 tests/integration/test_local_storage/files/example2.csv delete mode 100644 tests/integration/test_local_storage/test.py diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8bbb6a798ad..428986b5562 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -64,7 +64,6 @@ minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 oauthlib==3.2.0 -pandas==2.2.1 packaging==24.0 paramiko==3.4.0 pika==1.2.0 diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6637989202c..8f73e10c44f 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -335,16 +75,7 @@ static std::initializer_listgetOrSet( - cache_key, file_offset_of_buffer_end, size, file_size.value(), - create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); } - return !file_segments->empty(); } @@ -161,8 +158,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current read range: [{}, {})", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); + "Having {} file segments to read: {}, current offset: {}", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); initialized = true; } @@ -1046,10 +1043,6 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; - - LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", - remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); - size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1062,8 +1055,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); + fmt::format("Expected {} <= {} (size: {}, read range: {})", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); } swap(*implementation_buffer); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 01eb3d86696..bb9761a3905 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -169,7 +169,6 @@ void ReadBufferFromRemoteFSGather::initialize() bool ReadBufferFromRemoteFSGather::nextImpl() { - // LOG_DEBU /// Find first available buffer that fits to given offset. if (!current_buf) initialize(); diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index a8d5bd1797e..9f1cb681f1a 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -85,8 +85,6 @@ private: size_t current_buf_idx = 0; SeekableReadBufferPtr current_buf; - std::deque buffer_cemetery_; - LoggerPtr log; }; diff --git a/src/Disks/IO/createReadBufferFromFileBase.cpp b/src/Disks/IO/createReadBufferFromFileBase.cpp index 76588bcb5b1..b132e25ac6b 100644 --- a/src/Disks/IO/createReadBufferFromFileBase.cpp +++ b/src/Disks/IO/createReadBufferFromFileBase.cpp @@ -1,15 +1,14 @@ -#include -#include #include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include +#include +#include +#include #include -#include "ReadBufferFromRemoteFSGather.h" #include "config.h" namespace ProfileEvents @@ -100,7 +99,6 @@ std::unique_ptr createReadBufferFromFileBase( } else if (settings.local_fs_method == LocalFSReadMethod::io_uring) { - #if USE_LIBURING auto & reader = getIOUringReaderOrThrow(); res = std::make_unique( @@ -146,9 +144,8 @@ std::unique_ptr createReadBufferFromFileBase( settings.local_throttler); } else - { throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown read method"); - } + return res; }; diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 98f0c8c9059..73f5335411f 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -101,7 +101,10 @@ public: * * Try to read after the end should throw an exception. */ - bool ALWAYS_INLINE eof() { return !hasPendingData() && !next(); } + bool ALWAYS_INLINE eof() + { + return !hasPendingData() && !next(); + } void ignore() { diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 123a629f395..c8603fccb86 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -48,20 +48,10 @@ public: ConfigurationPtr configuration = base_configuration->clone(); - try { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start1"); metadata = DataLakeMetadata::create(object_storage, base_configuration, context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish1"); - - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files create storage"), "Data file name: {}", data_file); - } - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); if (use_schema_from_metadata) schema_from_metadata = metadata->getTableSchema(); } @@ -90,20 +80,9 @@ public: const std::optional & format_settings_, ContextPtr local_context) { - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start2"); - auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish2"); - - auto schema_from_metadata = metadata->getTableSchema(); - auto data_files = metadata->getDataFiles(); - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files get structure"), "Data file name: {}", data_file); - } if (!schema_from_metadata.empty()) { return ColumnsDescription(std::move(schema_from_metadata)); @@ -111,7 +90,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->setPaths(data_files); + configuration->setPaths(metadata->getDataFiles()); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } @@ -120,33 +99,14 @@ public: void updateConfiguration(ContextPtr local_context) override { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start3"); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish3"); - if (current_metadata && *current_metadata == *new_metadata) return; - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 1"); - - current_metadata = std::move(new_metadata); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 2"); - - auto data_files = current_metadata->getDataFiles(); - - LOG_DEBUG(&Poco::Logger::get("Update conf"), "Kek 3"); - - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data files size: {}", data_files.size()); - for (auto & data_file : data_files) - { - LOG_DEBUG(&Poco::Logger::get("Data Files update conf"), "Data file name: {}", data_file); - } auto updated_configuration = base_configuration->clone(); - updated_configuration->setPaths(data_files); + updated_configuration->setPaths(current_metadata->getDataFiles()); updated_configuration->setPartitionColumns(current_metadata->getPartitionColumns()); Storage::configuration = updated_configuration; @@ -188,10 +148,7 @@ private: if (!current_metadata) { Storage::updateConfiguration(local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Start4"); - current_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - LOG_DEBUG(&Poco::Logger::get("Metadata creating"), "Finish4"); } auto column_mapping = current_metadata->getColumnNameToPhysicalNameMapping(); if (!column_mapping.empty()) diff --git a/tests/integration/helpers/cloud_tools.py b/tests/integration/helpers/s3_tools.py similarity index 100% rename from tests/integration/helpers/cloud_tools.py rename to tests/integration/helpers/s3_tools.py diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py index f96f8acfaaf..0b7179c3cc4 100644 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ b/tests/integration/test_iceberg_azure_storage/test.py @@ -37,7 +37,7 @@ from pyspark.sql.window import Window from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject -from tests.integration.helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_local_storage/__init__.py b/tests/integration/test_local_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_local_storage/configs/config.xml b/tests/integration/test_local_storage/configs/config.xml deleted file mode 100644 index b4179ee51df..00000000000 --- a/tests/integration/test_local_storage/configs/config.xml +++ /dev/null @@ -1,3 +0,0 @@ - - /var/lib/clickhouse/ - diff --git a/tests/integration/test_local_storage/files/example2.csv b/tests/integration/test_local_storage/files/example2.csv deleted file mode 100644 index 7b6e6d6bab1..00000000000 --- a/tests/integration/test_local_storage/files/example2.csv +++ /dev/null @@ -1,3 +0,0 @@ -id,data -1,Str1 -2,Str2 diff --git a/tests/integration/test_local_storage/test.py b/tests/integration/test_local_storage/test.py deleted file mode 100644 index 89c3c17e1f0..00000000000 --- a/tests/integration/test_local_storage/test.py +++ /dev/null @@ -1,148 +0,0 @@ -import logging -import os -import random -import string - -import pytest - -from helpers.cluster import ClickHouseCluster - -from pathlib import Path - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "test_local_storage", main_configs=["configs/config.xml"], stay_alive=True - ) - - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def process_result(line: str): - return sorted( - list( - map( - lambda x: (int(x.split("\t")[0]), x.split("\t")[1]), - filter(lambda x: len(x) > 0, line.split("\n")), - ) - ) - ) - - -def test_local_engine(started_cluster): - node = started_cluster.instances["test_local_storage"] - node.query( - """ - CREATE TABLE test_0 ( - id Int64, - data String - ) ENGINE=Local('/data/example.csv', 'CSV'); - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """ - ) - - result = node.query( - """ - select * from test_0; - """ - ) - - assert [(-1, "7"), (1, "3"), (4, "abc")] == process_result(result) - - error_got = node.query_and_get_error( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """ - ) - - node.query( - """ - SET engine_file_truncate_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (5, 'arr'), (9, 'ty'), (0, '15'); - """, - settings={"engine_file_truncate_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - node.query( - """ - SET local_create_new_file_on_insert = 1; - """ - ) - - node.query( - """ - INSERT INTO test_0 VALUES (1, '3'), (-1, '7'), (4, 'abc'); - """, - settings={"local_create_new_file_on_insert": 1}, - ) - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [ - (-1, "7"), - (0, "15"), - (1, "3"), - (4, "abc"), - (5, "arr"), - (9, "ty"), - ] == process_result(result) - - node.restart_clickhouse() - - result = node.query( - """ - SELECT * FROM test_0; - """ - ) - - assert [(0, "15"), (5, "arr"), (9, "ty")] == process_result(result) - - -def test_table_function(started_cluster): - node = started_cluster.instances["test_local_storage"] - - node.copy_file_to_container( - "test_local_storage/files/example2.csv", "/data/example2.csv" - ) - - result = node.query( - """ - SELECT * FROM local('/data/example2.csv', 'CSV', 'id Int64, data String'); - """ - ) - - print("Res5", result) - - assert [(1, "Str1"), (2, "Str2")] == process_result(result) - - # assert False diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 1c1a7decdc3..35fe9d1bd6e 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 750b77b29f3..95825c2cd17 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from helpers.cloud_tools import prepare_s3_bucket, upload_directory, get_file_contents +from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b1d9b7f66bf..013a016efcd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -38,7 +38,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from helpers.cloud_tools import ( +from tests.integration.helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 31b1584507d..1c590824fd9 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from helpers.cloud_tools import prepare_s3_bucket +from tests.integration.helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 411e8f7cfb73d39df0614611766c3946603979ea Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:44:22 +0000 Subject: [PATCH 20/33] Remove unnecessary changes --- tests/integration/helpers/cluster.py | 1 - .../test_iceberg_azure_storage/__init__.py | 0 .../configs/config.d/named_collections.xml | 9 - .../configs/users.d/users.xml | 9 - .../test_iceberg_azure_storage/test.py | 291 ------------------ .../test_storage_azure_blob_storage/test.py | 2 - tests/integration/test_storage_delta/test.py | 2 +- tests/integration/test_storage_hudi/test.py | 2 +- tests/integration/test_storage_s3/test.py | 2 +- 9 files changed, 3 insertions(+), 315 deletions(-) delete mode 100644 tests/integration/test_iceberg_azure_storage/__init__.py delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml delete mode 100644 tests/integration/test_iceberg_azure_storage/test.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7e85bcf2f3c..215718463e8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2204,7 +2204,6 @@ class ClickHouseCluster: data = fdata.read() encodedBytes = base64.b64encode(data) encodedStr = str(encodedBytes, "utf-8") - self.exec_in_container( container_id, [ diff --git a/tests/integration/test_iceberg_azure_storage/__init__.py b/tests/integration/test_iceberg_azure_storage/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml b/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml deleted file mode 100644 index d4c54e2d13d..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/config.d/named_collections.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - http://minio1:9001/root/ - minio - minio123 - - - diff --git a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml b/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml deleted file mode 100644 index 4b6ba057ecb..00000000000 --- a/tests/integration/test_iceberg_azure_storage/configs/users.d/users.xml +++ /dev/null @@ -1,9 +0,0 @@ - - - - - default - 1 - - - diff --git a/tests/integration/test_iceberg_azure_storage/test.py b/tests/integration/test_iceberg_azure_storage/test.py deleted file mode 100644 index 0b7179c3cc4..00000000000 --- a/tests/integration/test_iceberg_azure_storage/test.py +++ /dev/null @@ -1,291 +0,0 @@ -import helpers.client -from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.test_tools import TSV - -import pyspark -import logging -import os -import json -import pytest -import time -import glob -import uuid -import os - -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - -from pyspark.sql.types import ( - StructType, - StructField, - StringType, - IntegerType, - DateType, - TimestampType, - BooleanType, - ArrayType, -) -from pyspark.sql.functions import current_timestamp -from datetime import datetime -from pyspark.sql.functions import monotonically_increasing_id, row_number -from pyspark.sql.window import Window -from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 -from minio.deleteobjects import DeleteObject - -from tests.integration.helpers.s3_tools import ( - prepare_s3_bucket, - upload_directory, - get_file_contents, - list_s3_objects, -) - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - - -def get_spark(): - builder = ( - pyspark.sql.SparkSession.builder.appName("spark_test") - .config( - "spark.sql.catalog.spark_catalog", - "org.apache.iceberg.spark.SparkSessionCatalog", - ) - .config("spark.sql.catalog.local", "org.apache.iceberg.spark.SparkCatalog") - .config("spark.sql.catalog.spark_catalog.type", "hadoop") - .config("spark.sql.catalog.spark_catalog.warehouse", "/iceberg_data") - .config( - "spark.sql.extensions", - "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions", - ) - .master("local") - ) - return builder.master("local").getOrCreate() - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__, with_spark=True) - cluster.add_instance( - "node1", - main_configs=["configs/config.d/named_collections.xml"], - user_configs=["configs/users.d/users.xml"], - with_minio=True, - stay_alive=True, - ) - - logging.info("Starting cluster...") - cluster.start() - - prepare_s3_bucket(cluster) - logging.info("S3 bucket created") - - cluster.spark_session = get_spark() - - yield cluster - - finally: - cluster.shutdown() - - -def run_query(instance, query, stdin=None, settings=None): - # type: (ClickHouseInstance, str, object, dict) -> str - - logging.info("Running query '{}'...".format(query)) - result = instance.query(query, stdin=stdin, settings=settings) - logging.info("Query finished") - - return result - - -def write_iceberg_from_file( - spark, path, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - spark.read.load(f"file://{path}").writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).partitionedBy( - partition_by - ).tableProperty("format-version", format_version).using("iceberg").create() - else: - spark.read.load(f"file://{path}").writeTo(table_name).append() - - -def write_iceberg_from_df( - spark, df, table_name, mode="overwrite", format_version="1", partition_by=None -): - if mode == "overwrite": - if partition_by is None: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).using("iceberg").create() - else: - df.writeTo(table_name).tableProperty( - "format-version", format_version - ).partitionedBy(partition_by).using("iceberg").create() - else: - df.writeTo(table_name).append() - - -def generate_data(spark, start, end): - a = spark.range(start, end, 1).toDF("a") - b = spark.range(start + 1, end + 1, 1).toDF("b") - b = b.withColumn("b", b["b"].cast(StringType())) - - a = a.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - b = b.withColumn( - "row_index", row_number().over(Window.orderBy(monotonically_increasing_id())) - ) - - df = a.join(b, on=["row_index"]).drop("row_index") - return df - - -def create_iceberg_table(node, table_name, format="Parquet", bucket="root"): - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=Iceberg(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) - - -def create_initial_data_file( - cluster, node, query, table_name, compression_method="none" -): - node.query( - f""" - INSERT INTO TABLE FUNCTION - file('{table_name}.parquet') - SETTINGS - output_format_parquet_compression_method='{compression_method}', - s3_truncate_on_insert=1 {query} - FORMAT Parquet""" - ) - user_files_path = os.path.join( - SCRIPT_DIR, f"{cluster.instances_dir_name}/node1/database/user_files" - ) - result_path = f"{user_files_path}/{table_name}.parquet" - return result_path - - -@pytest.mark.parametrize("format_version", ["1", "2"]) -def test_single_iceberg_file(started_cluster, format_version): - instance = started_cluster.instances["node1"] - spark = started_cluster.spark_session - minio_client = started_cluster.minio_client - bucket = started_cluster.minio_bucket - TABLE_NAME = "test_single_iceberg_file_" + format_version - - inserted_data = "SELECT number, toString(number) as string FROM numbers(100)" - parquet_data_path = create_initial_data_file( - started_cluster, instance, inserted_data, TABLE_NAME - ) - - write_iceberg_from_file( - spark, parquet_data_path, TABLE_NAME, format_version=format_version - ) - - files = upload_directory( - minio_client, bucket, f"/iceberg_data/default/{TABLE_NAME}/", "" - ) - - for bucket in minio_client.list_buckets(): - for object in minio_client.list_objects(bucket.name, recursive=True): - print("Object: ", object.object_name) - extension = object.object_name.split(".")[-1] - print("File extension: ", extension) - try: - response = minio_client.get_object( - object.bucket_name, object.object_name - ) - - if extension == "avro": - avro_bytes = response.read() - - # Use BytesIO to create a file-like object from the byte string - avro_file = io.BytesIO(avro_bytes) - - # Read the Avro data - reader = avro.datafile.DataFileReader( - avro_file, avro.io.DatumReader() - ) - records = [record for record in reader] - - # Close the reader - reader.close() - - # Now you can work with the records - for record in records: - # print(json.dumps(record, indent=4, sort_keys=True)) - print(str(record)) - # my_json = ( - # str(record) - # .replace("'", '"') - # .replace("None", "null") - # .replace('b"', '"') - # ) - # print(my_json) - # data = json.loads(my_json) - # s = json.dumps(data, indent=4, sort_keys=True) - # print(s) - elif extension == "json": - my_bytes_value = response.read() - my_json = my_bytes_value.decode("utf8").replace("'", '"') - data = json.loads(my_json) - s = json.dumps(data, indent=4, sort_keys=True) - print(s) - elif extension == "parquet": - # print("To be continued...") - # # Your byte string containing the Parquet data - # parquet_bytes = response.read() - - # # Use BytesIO to create a file-like object from the byte string - # parquet_file = io.BytesIO(parquet_bytes) - - # # Read the Parquet data into a PyArrow Table - # table = pq.read_table(parquet_file) - - # # Convert the PyArrow Table to a Pandas DataFrame - # df = table.to_pandas() - - # # Now you can work with s DataFrame - # print(df) - parquet_bytes = ( - response.read() - ) # Replace with your actual byte string - - # Create a temporary file and write the byte string to it - with tempfile.NamedTemporaryFile(delete=False) as tmp_file: - tmp_file.write(parquet_bytes) - tmp_file_path = tmp_file.name - - # Read the Parquet file using PySpark - df = spark.read.parquet(tmp_file_path) - - # Show the DataFrame - print(df.toPandas()) - else: - print(response.read()) - - finally: - print("----------------") - response.close() - response.release_conn() - - create_iceberg_table(instance, TABLE_NAME) - - assert instance.query(f"SELECT * FROM {TABLE_NAME}") == instance.query( - inserted_data - ) - - assert 0 == 1 diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index c5599d20519..6fbe7634642 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -550,7 +550,6 @@ def test_schema_inference_from_globs(cluster): def test_simple_write_account_string_table_function(cluster): node = cluster.instances["node"] port = cluster.env_variables["AZURITE_PORT"] - print("Account URL: ", cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]) azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('{cluster.env_variables['AZURITE_STORAGE_ACCOUNT_URL']}', " @@ -560,7 +559,6 @@ def test_simple_write_account_string_table_function(cluster): ) print(get_azure_file_content("test_simple_write_tf.csv", port)) assert get_azure_file_content("test_simple_write_tf.csv", port) == '1,"a"\n' - assert 0 == 1 def test_simple_write_connection_string_table_function(cluster): diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 35fe9d1bd6e..054b79ff6fe 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -30,7 +30,7 @@ from pyspark.sql.functions import monotonically_increasing_id, row_number from pyspark.sql.window import Window from minio.deleteobjects import DeleteObject -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, upload_directory, get_file_contents, diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 95825c2cd17..0c3fbfb3cda 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -6,7 +6,7 @@ import json import helpers.client from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.test_tools import TSV -from tests.integration.helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents +from helpers.s3_tools import prepare_s3_bucket, upload_directory, get_file_contents import pyspark from pyspark.sql.types import ( diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1c590824fd9..ab327afe90b 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -13,7 +13,7 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.mock_servers import start_mock_servers from helpers.test_tools import exec_query_with_retry -from tests.integration.helpers.s3_tools import prepare_s3_bucket +from helpers.s3_tools import prepare_s3_bucket MINIO_INTERNAL_PORT = 9001 From 42d20f2a8d34640233c2dc4002d5fe611f2f9c77 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 12:52:11 +0000 Subject: [PATCH 21/33] Remove table registration --- .../StorageObjectStorageSource.cpp | 2 -- .../registerStorageObjectStorage.cpp | 26 ++++--------------- .../TableFunctionObjectStorage.cpp | 17 ++++-------- .../TableFunctionObjectStorage.h | 24 ++++------------- 4 files changed, 15 insertions(+), 54 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 74fcd0418ab..ed68dd61967 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -437,8 +437,6 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer( { LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - LOG_DEBUG(&Poco::Logger::get("Get path"), "Path: {}", object_info.getPath()); - auto async_reader = object_storage->readObjects( StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 4bc8cfa6a2f..b5f4cf5bb54 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,15 +1,16 @@ #include #include #include -#include -#include #include +#include #include #include namespace DB { +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS + namespace ErrorCodes { extern const int BAD_ARGUMENTS; @@ -69,6 +70,8 @@ static std::shared_ptr createStorageObjectStorage( partition_by); } +#endif + #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { @@ -136,27 +139,8 @@ void registerStorageHDFS(StorageFactory & factory) } #endif -void registerStorageLocal(StorageFactory & factory) -{ - factory.registerStorage( - "Local", - [=](const StorageFactory::Arguments & args) - { - auto configuration = std::make_shared(); - return createStorageObjectStorage(args, configuration, args.getLocalContext()); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::FILE, - }); -} - - void registerStorageObjectStorage(StorageFactory & factory) { - registerStorageLocal(factory); #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index d156afb81a1..550d9cc799b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -12,13 +12,12 @@ #include -#include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include namespace DB @@ -204,11 +203,6 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); #endif - factory.registerFunction>( - {.documentation - = {.description = R"(The table function can be used to read the data stored locally.)", - .examples{{"local", "SELECT * FROM local(file_path, [format, compression, structure])", ""}}}, - .allow_readonly = false}); } #if USE_AZURE_BLOB_STORAGE @@ -229,5 +223,4 @@ template class TableFunctionObjectStorage; #endif -template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 3468e5c5007..86b8f0d5e14 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include -#include -#include -#include -#include #include "config.h" +#include +#include +#include +#include +#include namespace DB { @@ -14,7 +14,6 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; -class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -91,17 +90,6 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; -struct LocalDefinition -{ - static constexpr auto name = "local"; - static constexpr auto storage_type_name = "Local"; - static constexpr auto signature = " - path\n" - " - path, format\n" - " - path, format, structure\n" - " - path, format, structure, compression_method\n"; - static constexpr auto max_number_of_arguments = 4; -}; - template class TableFunctionObjectStorage : public ITableFunction { @@ -181,6 +169,4 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif - -using TableFunctionLocal = TableFunctionObjectStorage; } From 58dd01d8b61e76c378a45bc84967b990bf355e4e Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 13:45:00 +0000 Subject: [PATCH 22/33] Add docs --- .../table-engines/integrations/iceberg.md | 30 +++++++++++----- .../sql-reference/table-functions/iceberg.md | 34 +++++++++++-------- .../integration/test_storage_iceberg/test.py | 10 +----- 3 files changed, 42 insertions(+), 32 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 21fdbc0b1a5..40f3e7c22fd 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -6,28 +6,34 @@ sidebar_label: Iceberg # Iceberg Table Engine -This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +This engine provides a read-only integration with existing Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure and locally stored tables. ## Create Table -Note that the Iceberg table must already exist in S3, this command does not take DDL parameters to create a new table. +Note that the Iceberg table must already exist in the storage, this command does not take DDL parameters to create a new table. ``` sql -CREATE TABLE iceberg_table - ENGINE = Iceberg(url, [aws_access_key_id, aws_secret_access_key,]) +CREATE TABLE iceberg_table_s3 + ENGINE = IcebergS3(url, [, NOSIGN | access_key_id, secret_access_key, [session_token]], format, [,compression]) + +CREATE TABLE iceberg_table_azure + ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) + +CREATE TABLE iceberg_table_local + ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) ``` -**Engine parameters** +**Engine arguments** -- `url` — url with the path to an existing Iceberg table. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. +Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +`format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) **Example** ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +CREATE TABLE iceberg_table ENGINE=IcebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` Using named collections: @@ -45,9 +51,15 @@ Using named collections: ``` ```sql -CREATE TABLE iceberg_table ENGINE=Iceberg(iceberg_conf, filename = 'test_table') +CREATE TABLE iceberg_table ENGINE=IcebergS3(iceberg_conf, filename = 'test_table') + ``` +**Aliases** + + +Table engine `Iceberg` is an alias to `IcebergS3` now. + ## See also - [iceberg table function](/docs/en/sql-reference/table-functions/iceberg.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index fa86b436a5e..dfe8d6f2d0a 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -6,35 +6,37 @@ sidebar_label: iceberg # iceberg Table Function -Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3. +Provides a read-only table-like interface to Apache [Iceberg](https://iceberg.apache.org/) tables in Amazon S3, Azure or locally stored. ## Syntax ``` sql -iceberg(url [,aws_access_key_id, aws_secret_access_key] [,format] [,structure]) +icebergS3(url [, NOSIGN | access_key_id, secret_access_key, [session_token]] [,format] [,compression_method]) +icebergS3(named_collection[, option=value [,..]]) + +icebergAzure(connection_string|storage_account_url, container_name, blobpath, [,account_name], [,account_key] [,format] [,compression_method]) +icebergAzure(named_collection[, option=value [,..]]) + +icebergLocal(path_to_table, [,format] [,compression_method]) +icebergLocal(named_collection[, option=value [,..]]) ``` ## Arguments -- `url` — Bucket url with the path to an existing Iceberg table in S3. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. These parameters are optional. If credentials are not specified, they are used from the ClickHouse configuration. For more information see [Using S3 for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3). -- `format` — The [format](/docs/en/interfaces/formats.md/#formats) of the file. By default `Parquet` is used. -- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. - -Engine parameters can be specified using [Named Collections](/docs/en/operations/named-collections.md). +Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +`format` stands for the format of data files in the Iceberg table. **Returned value** - -A table with the specified structure for reading data in the specified Iceberg table in S3. +A table with the specified structure for reading data in the specified Iceberg table. **Example** ```sql -SELECT * FROM iceberg('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') +SELECT * FROM icebergS3('http://test.s3.amazonaws.com/clickhouse-bucket/test_table', 'test', 'test') ``` :::important -ClickHouse currently supports reading v1 (v2 support is coming soon!) of the Iceberg format via the `iceberg` table function and `Iceberg` table engine. +ClickHouse currently supports reading v1 and v2 of the Iceberg format via the `icebergS3`, `icebergAzure` and `icebergLocal` table functions and `IcebergS3`, `icebergAzure` ans `icebergLocal` table engines. ::: ## Defining a named collection @@ -56,10 +58,14 @@ Here is an example of configuring a named collection for storing the URL and cre ``` ```sql -SELECT * FROM iceberg(iceberg_conf, filename = 'test_table') -DESCRIBE iceberg(iceberg_conf, filename = 'test_table') +SELECT * FROM icebergS3(iceberg_conf, filename = 'test_table') +DESCRIBE icebergS3(iceberg_conf, filename = 'test_table') ``` +**Aliases** + +Table function `iceberg` is an alias to `icebergS3` now. + **See Also** - [Iceberg engine](/docs/en/engines/table-engines/integrations/iceberg.md) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index 013a016efcd..b66133f5562 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -12,14 +12,6 @@ import glob import uuid import os -import tempfile - -import io -import avro.schema -import avro.io -import avro.datafile -import pandas as pd - from pyspark.sql.types import ( StructType, StructField, @@ -38,7 +30,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from minio.deleteobjects import DeleteObject from azure.storage.blob import BlobServiceClient -from tests.integration.helpers.s3_tools import ( +from helpers.s3_tools import ( prepare_s3_bucket, get_file_contents, list_s3_objects, From 61e616204eee76289e3da63dd663ce39a0316d0c Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 14:21:42 +0000 Subject: [PATCH 23/33] Docs correct typos --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- docs/en/sql-reference/table-functions/iceberg.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 40f3e7c22fd..78e076a7427 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -25,7 +25,7 @@ CREATE TABLE iceberg_table_local **Engine arguments** -Description of the arguments coincides with decription of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. +Description of the arguments coincides with description of arguments in engines `S3`, `AzureBlobStorage` and `File` correspondingly. `format` stands for the format of data files in the Iceberg table. Engine parameters can be specified using [Named Collections](../../../operations/named-collections.md) diff --git a/docs/en/sql-reference/table-functions/iceberg.md b/docs/en/sql-reference/table-functions/iceberg.md index dfe8d6f2d0a..784fd646860 100644 --- a/docs/en/sql-reference/table-functions/iceberg.md +++ b/docs/en/sql-reference/table-functions/iceberg.md @@ -23,7 +23,7 @@ icebergLocal(named_collection[, option=value [,..]]) ## Arguments -Description of the arguments coincides with decription of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. +Description of the arguments coincides with description of arguments in table functions `s3`, `azureBlobStorage` and `file` correspondingly. `format` stands for the format of data files in the Iceberg table. **Returned value** From 9c25314a58721c18c265e46276199308ddc6cce0 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 15:28:31 +0000 Subject: [PATCH 24/33] Fixed settings --- src/Core/SettingsChangesHistory.cpp | 12 +++++++++++- src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp | 17 ++++++++++++----- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8f73e10c44f..86988d697b7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -75,7 +75,17 @@ static std::initializer_listgetOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet( + cache_key, file_offset_of_buffer_end, size, file_size.value(), + create_settings, settings.filesystem_cache_segments_batch_size, user); } + return !file_segments->empty(); } @@ -158,8 +161,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current offset: {}", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + "Having {} file segments to read: {}, current read range: [{}, {})", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); initialized = true; } @@ -1043,6 +1046,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; + + LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", + remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); + size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1055,8 +1062,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {})", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); + fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); } swap(*implementation_buffer); From 25ce6df8957c36358498aed83ce2e17d034261f3 Mon Sep 17 00:00:00 2001 From: divanik Date: Mon, 12 Aug 2024 16:22:30 +0000 Subject: [PATCH 25/33] Return local table --- .../TableFunctionObjectStorage.cpp | 2 +- .../TableFunctionObjectStorage.h | 22 +++++++++++++++---- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 550d9cc799b..6235108e2ba 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -222,5 +222,5 @@ template class TableFunctionObjectStorage template class TableFunctionObjectStorage; template class TableFunctionObjectStorage; #endif - +template class TableFunctionObjectStorage; } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 86b8f0d5e14..3468e5c5007 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,11 +1,11 @@ #pragma once -#include "config.h" -#include -#include #include -#include +#include #include +#include +#include +#include "config.h" namespace DB { @@ -14,6 +14,7 @@ class Context; class StorageS3Configuration; class StorageAzureConfiguration; class StorageHDFSConfiguration; +class StorageLocalConfiguration; struct S3StorageSettings; struct AzureStorageSettings; struct HDFSStorageSettings; @@ -90,6 +91,17 @@ struct HDFSDefinition static constexpr auto max_number_of_arguments = 4; }; +struct LocalDefinition +{ + static constexpr auto name = "local"; + static constexpr auto storage_type_name = "Local"; + static constexpr auto signature = " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; +}; + template class TableFunctionObjectStorage : public ITableFunction { @@ -169,4 +181,6 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif + +using TableFunctionLocal = TableFunctionObjectStorage; } From 534ec1fa01b9de4206a8d6e3f91d73d7d1b2f743 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 10:12:45 +0000 Subject: [PATCH 26/33] Correct compile error --- src/TableFunctions/TableFunctionObjectStorage.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 6235108e2ba..0ddf901ee21 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -14,10 +14,11 @@ #include #include -#include -#include -#include #include +#include +#include +#include +#include namespace DB From be92986eda3298f7f44d64509bda83689713a2a0 Mon Sep 17 00:00:00 2001 From: divanik Date: Tue, 13 Aug 2024 14:48:29 +0000 Subject: [PATCH 27/33] Possibly fix broken tests and build --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp | 2 +- src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h | 2 +- src/TableFunctions/registerDataLakeTableFunctions.cpp | 2 -- tests/integration/helpers/s3_tools.py | 2 +- 6 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c896a760597..949f994fa09 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -3,7 +3,7 @@ #include "config.h" #include -#if USE_AWS_S3 && USE_PARQUET +#if USE_PARQUET #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index c8603fccb86..ae0e3066ffb 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -2,7 +2,7 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 6d18b13df01..9b9d92e282c 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -1,6 +1,6 @@ #include "config.h" -#if USE_AWS_S3 && USE_AVRO +#if USE_AVRO #include #include diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index 9476ac6e7d9..7b0deab91c3 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -1,6 +1,6 @@ #pragma once -#if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. #include #include diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp index 8de852f4fba..8361d8a7977 100644 --- a/src/TableFunctions/registerDataLakeTableFunctions.cpp +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -75,10 +75,8 @@ void registerDataLakeTableFunctions(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AVRO -# if USE_AWS_S3 registerTableFunctionIceberg(factory); #endif -#endif #if USE_AWS_S3 # if USE_PARQUET registerTableFunctionDeltaLake(factory); diff --git a/tests/integration/helpers/s3_tools.py b/tests/integration/helpers/s3_tools.py index 39c56e0eb85..5b727060e69 100644 --- a/tests/integration/helpers/s3_tools.py +++ b/tests/integration/helpers/s3_tools.py @@ -85,7 +85,7 @@ class AzureUploader(CloudUploader): def upload_directory(minio_client, bucket, local_path, remote_path): - S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( + return S3Uploader(minio_client=minio_client, bucket_name=bucket).upload_directory( local_path, remote_path ) From 476819a56fd71e445da5aa91f34e10abb8ded327 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 14 Aug 2024 11:35:37 +0000 Subject: [PATCH 28/33] Changed test to work in parallel --- .../integration/test_storage_iceberg/test.py | 130 +++++++++++++----- 1 file changed, 97 insertions(+), 33 deletions(-) diff --git a/tests/integration/test_storage_iceberg/test.py b/tests/integration/test_storage_iceberg/test.py index b66133f5562..176c7e209bd 100644 --- a/tests/integration/test_storage_iceberg/test.py +++ b/tests/integration/test_storage_iceberg/test.py @@ -165,9 +165,8 @@ def generate_data(spark, start, end): return df -def create_iceberg_table( +def get_creation_expression( storage_type, - node, table_name, cluster, format="Parquet", @@ -182,38 +181,52 @@ def create_iceberg_table( print(bucket) if table_function: return f"icebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')" - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergS3(s3, filename = 'iceberg_data/default/{table_name}/', format={format}, url = 'http://minio1:9001/{bucket}/')""" elif storage_type == "azure": if table_function: return f""" icebergAzure(azure, container = '{cluster.azure_container_name}', storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergAzure(azure, container = {cluster.azure_container_name}, storage_account_url = '{cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]}', blob_path = '/iceberg_data/default/{table_name}/', format={format})""" elif storage_type == "local": if table_function: return f""" icebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format}) """ - node.query( - f""" - DROP TABLE IF EXISTS {table_name}; - CREATE TABLE {table_name} - ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" - ) + else: + return f""" + DROP TABLE IF EXISTS {table_name}; + CREATE TABLE {table_name} + ENGINE=IcebergLocal(local, path = '/iceberg_data/default/{table_name}/', format={format});""" else: raise Exception(f"Unknown iceberg storage type: {storage_type}") +def get_uuid_str(): + return str(uuid.uuid4()).replace("-", "_") + + +def create_iceberg_table( + storage_type, + node, + table_name, + cluster, + format="Parquet", + **kwargs, +): + node.query( + get_creation_expression(storage_type, table_name, cluster, format, **kwargs) + ) + + def create_initial_data_file( cluster, node, query, table_name, compression_method="none" ): @@ -258,7 +271,14 @@ def default_upload_directory( def test_single_iceberg_file(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_single_iceberg_file_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_single_iceberg_file_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df(spark, generate_data(spark, 0, 100), TABLE_NAME) @@ -281,7 +301,14 @@ def test_single_iceberg_file(started_cluster, format_version, storage_type): def test_partition_by(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_partition_by_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_partition_by_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -311,7 +338,14 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_multiple_iceberg_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_multiple_iceberg_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -364,7 +398,9 @@ def test_multiple_iceberg_files(started_cluster, format_version, storage_type): def test_types(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_types_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_types_" + format_version + "_" + storage_type + "_" + get_uuid_str() + ) data = [ ( @@ -404,8 +440,8 @@ def test_types(started_cluster, format_version, storage_type): == "123\tstring\t2000-01-01\t['str1','str2']\ttrue" ) - table_function_expr = create_iceberg_table( - storage_type, instance, TABLE_NAME, started_cluster, table_function=True + table_function_expr = get_creation_expression( + storage_type, TABLE_NAME, started_cluster, table_function=True ) assert ( instance.query(f"SELECT a, b, c, d, e FROM {table_function_expr}").strip() @@ -430,7 +466,14 @@ def test_delete_files(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_delete_files_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_delete_files_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -496,7 +539,14 @@ def test_evolved_schema(started_cluster, format_version, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_evolved_schema_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_evolved_schema_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) write_iceberg_from_df( spark, @@ -542,7 +592,7 @@ def test_row_based_deletes(started_cluster, storage_type): spark = started_cluster.spark_session minio_client = started_cluster.minio_client bucket = started_cluster.minio_bucket - TABLE_NAME = "test_row_based_deletes_" + storage_type + TABLE_NAME = "test_row_based_deletes_" + storage_type + "_" + get_uuid_str() spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -587,6 +637,8 @@ def test_schema_inference(started_cluster, format_version, storage_type): + format_version + "_" + storage_type + + "_" + + get_uuid_str() ) # Types time, timestamptz, fixed are not supported in Spark. @@ -645,7 +697,14 @@ def test_schema_inference(started_cluster, format_version, storage_type): def test_metadata_file_selection(started_cluster, format_version, storage_type): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_metadata_selection_" + format_version + "_" + storage_type + TABLE_NAME = ( + "test_metadata_selection_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() + ) spark.sql( f"CREATE TABLE {TABLE_NAME} (id bigint, data string) USING iceberg TBLPROPERTIES ('format-version' = '2', 'write.update.mode'='merge-on-read', 'write.delete.mode'='merge-on-read', 'write.merge.mode'='merge-on-read')" @@ -674,7 +733,12 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage instance = started_cluster.instances["node1"] spark = started_cluster.spark_session TABLE_NAME = ( - "test_metadata_selection_with_uuid_" + format_version + "_" + storage_type + "test_metadata_selection_with_uuid_" + + format_version + + "_" + + storage_type + + "_" + + get_uuid_str() ) spark.sql( @@ -689,7 +753,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage for i in range(50): os.rename( f"/iceberg_data/default/{TABLE_NAME}/metadata/v{i + 1}.metadata.json", - f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{uuid.uuid4()}.metadata.json", + f"/iceberg_data/default/{TABLE_NAME}/metadata/{str(i).zfill(5)}-{get_uuid_str()}.metadata.json", ) files = default_upload_directory( @@ -707,7 +771,7 @@ def test_metadata_file_format_with_uuid(started_cluster, format_version, storage def test_restart_broken_s3(started_cluster): instance = started_cluster.instances["node1"] spark = started_cluster.spark_session - TABLE_NAME = "test_restart_broken_table_function_s3" + TABLE_NAME = "test_restart_broken_table_function_s3" + "_" + get_uuid_str() minio_client = started_cluster.minio_client bucket = "broken2" From df07b07cb9c89200f51ffed4fffd17cb2b0307e8 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 12:18:02 +0000 Subject: [PATCH 29/33] Add new setting --- contrib/NuRaft | 2 +- contrib/azure | 2 +- contrib/icu | 2 +- contrib/icudata | 2 +- contrib/libprotobuf-mutator | 2 +- contrib/librdkafka | 2 +- contrib/libunwind | 2 +- contrib/numactl | 1 + contrib/qpl | 2 +- contrib/rocksdb | 2 +- contrib/usearch | 2 +- contrib/zlib-ng | 2 +- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 3 ++- src/Storages/ObjectStorage/Local/Configuration.cpp | 2 +- src/Storages/ObjectStorage/Utils.cpp | 2 +- 16 files changed, 17 insertions(+), 14 deletions(-) create mode 160000 contrib/numactl diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce diff --git a/contrib/azure b/contrib/azure index ea3e19a7be0..67272b7ee0a 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ea3e19a7be08519134c643177d56c7484dfec884 +Subproject commit 67272b7ee0adff6b69921b26eb071ba1a353062c diff --git a/contrib/icu b/contrib/icu index a56dde820dc..7750081bda4 160000 --- a/contrib/icu +++ b/contrib/icu @@ -1 +1 @@ -Subproject commit a56dde820dc35665a66f2e9ee8ba58e75049b668 +Subproject commit 7750081bda4b3bc1768ae03849ec70f67ea10625 diff --git a/contrib/icudata b/contrib/icudata index c8e717892a5..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit c8e717892a557b4d2852317c7d628aacc0a0e5ab +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index a304ec48dcf..1f95f808306 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit a304ec48dcf15d942607032151f7e9ee504b5dcf +Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d diff --git a/contrib/librdkafka b/contrib/librdkafka index 2d2aab6f5b7..39d4ed49ccf 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 2d2aab6f5b79db1cfca15d7bf0dee75d00d82082 +Subproject commit 39d4ed49ccf3406e2bf825d5d7b0903b5a290782 diff --git a/contrib/libunwind b/contrib/libunwind index 8f28e64d158..a89d904befe 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 +Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 diff --git a/contrib/numactl b/contrib/numactl new file mode 160000 index 00000000000..8d13d63a05f --- /dev/null +++ b/contrib/numactl @@ -0,0 +1 @@ +Subproject commit 8d13d63a05f0c3cd88bf777cbb61541202b7da08 diff --git a/contrib/qpl b/contrib/qpl index d4715e0e798..c2ced94c53c 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit d4715e0e79896b85612158e135ee1a85f3b3e04d +Subproject commit c2ced94c53c1ee22191201a59878e9280bc9b9b8 diff --git a/contrib/rocksdb b/contrib/rocksdb index be366233921..5f003e4a22d 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit be366233921293bd07a84dc4ea6991858665f202 +Subproject commit 5f003e4a22d2e48e37c98d9620241237cd30dd24 diff --git a/contrib/usearch b/contrib/usearch index 955c6f9c11a..30810452bec 160000 --- a/contrib/usearch +++ b/contrib/usearch @@ -1 +1 @@ -Subproject commit 955c6f9c11adfd89c912e0d1643d160b4e9e543f +Subproject commit 30810452bec5d3d3aa0931bb5d761e2f09aa6356 diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 50f0eae1a41..a2fbeffdc30 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 50f0eae1a411764cd6d1e85b3ce471438acd3c1c +Subproject commit a2fbeffdc30a8b0ce6d54ee31208e2688eac4c9f diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b19ef1000c1..acc02cc4b92 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -113,6 +113,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ + M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7c163750a80..7d1aab939f8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -81,7 +81,8 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.engine_file_truncate_on_insert, + .truncate_on_insert = settings.local_truncate_on_insert, .create_new_file_on_insert = settings.local_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 6491deef440..3836e2c7ec8 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -42,7 +42,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", configuration.getNamespace(), key, - configuration.getTypeName() == "local" ? "engine_file" : configuration.getTypeName(), + configuration.getTypeName(), configuration.getTypeName()); } From 0b58cbca3836a9dbf2babb9a0c2d7ed89d44b90b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:02:25 +0000 Subject: [PATCH 30/33] Fix submodules --- .gitmodules | 3 +++ contrib/usearch | 1 + 2 files changed, 4 insertions(+) create mode 160000 contrib/usearch diff --git a/.gitmodules b/.gitmodules index 26778cda171..cdee6a43ad8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -336,6 +336,9 @@ [submodule "contrib/incbin"] path = contrib/incbin url = https://github.com/graphitemaster/incbin.git +[submodule "contrib/usearch"] + path = contrib/usearch + url = https://github.com/ClickHouse/usearch.git [submodule "contrib/SimSIMD"] path = contrib/SimSIMD url = https://github.com/ashvardanian/SimSIMD.git diff --git a/contrib/usearch b/contrib/usearch new file mode 160000 index 00000000000..e21a5778a0d --- /dev/null +++ b/contrib/usearch @@ -0,0 +1 @@ +Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4 From 53bff26f11301781f92a3005c9a08fe176ff0bda Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:09:39 +0000 Subject: [PATCH 31/33] Try to fix submodule --- contrib/libprotobuf-mutator | 2 +- contrib/libunwind | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/libprotobuf-mutator b/contrib/libprotobuf-mutator index 1f95f808306..b922c8ab900 160000 --- a/contrib/libprotobuf-mutator +++ b/contrib/libprotobuf-mutator @@ -1 +1 @@ -Subproject commit 1f95f8083066f5b38fd2db172e7e7f9aa7c49d2d +Subproject commit b922c8ab9004ef9944982e4f165e2747b13223fa diff --git a/contrib/libunwind b/contrib/libunwind index a89d904befe..601db0b0e03 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit a89d904befea07814628c6ce0b44083c4e149c62 +Subproject commit 601db0b0e03018c01710470a37703b618f9cf08b From ef1f0e2aafc68d312cfc793aa4652e81e58c199b Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 13:16:09 +0000 Subject: [PATCH 32/33] Fix typo in docs --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 78e076a7427..94468066372 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -20,7 +20,7 @@ CREATE TABLE iceberg_table_azure ENGINE = IcebergAzure(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) CREATE TABLE iceberg_table_local - ENGINE = ICebergLocal(path_to_table, [,format] [,compression_method]) + ENGINE = IcebergLocal(path_to_table, [,format] [,compression_method]) ``` **Engine arguments** From f17b70e3f3bea09dc09c4b22d857fbab4e1417e4 Mon Sep 17 00:00:00 2001 From: divanik Date: Thu, 15 Aug 2024 14:06:00 +0000 Subject: [PATCH 33/33] Resolve issues with settings --- src/Core/Settings.h | 2 - src/Core/SettingsChangesHistory.cpp | 262 ------------------ .../ObjectStorage/Local/Configuration.cpp | 4 +- src/Storages/ObjectStorage/Utils.cpp | 5 +- 4 files changed, 3 insertions(+), 270 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19a59bf7777..6bca8a99f02 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -116,9 +116,7 @@ class IColumn; M(UInt64, azure_list_object_keys_size, 1000, "Maximum number of files that could be returned in batch by ListObject request", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, azure_truncate_on_insert, false, "Enables or disables truncate before insert in azure engine tables.", 0) \ - M(Bool, local_truncate_on_insert, false, "Enables or disables truncate before insert in local engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ - M(Bool, local_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in local object storage engine tables", 0) \ M(Bool, s3_skip_empty_files, false, "Allow to skip empty files in s3 table engine", 0) \ M(Bool, azure_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in azure engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6f73ca4059f..815880eb63e 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,267 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"optimize_functions_to_subcolumns", false, true, "Enable optimization by default"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_ignore_key_case", false, false, "Ignore json key case while read json field from string."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_read_use_writer_time_zone", false, false, "Whether use the writer's time zone in ORC stripe for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."}, - {"local_create_new_file_on_insert", false, false, "Enables or disables creating a new file on each insert in local object storage engine tables"}, - {"local_truncate_on_insert", false, false, "Enables or disables truncating the file on each insert in local object storage engine tables"}, - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -350,7 +89,6 @@ static std::initializer_listgetSettingsRef(); return StorageObjectStorage::QuerySettings{ - .truncate_on_insert = settings.local_truncate_on_insert, - .create_new_file_on_insert = settings.local_create_new_file_on_insert, + .truncate_on_insert = settings.engine_file_truncate_on_insert, + .create_new_file_on_insert = false, .schema_inference_use_cache = settings.schema_inference_use_cache_for_file, .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.engine_file_skip_empty_files, diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 77725f8ed18..73410d959e0 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -40,10 +40,7 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", - configuration.getNamespace(), - key, - configuration.getTypeName(), - configuration.getTypeName()); + configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); } void resolveSchemaAndFormat(