diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index afaa189701d..5714fef8347 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -481,6 +481,11 @@ if (ARCH_S390X) else() set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" ) endif() +# ^^ you might be confused how for different little endian platforms (x86, ARM) the same assembly files can be used. +# These files are indeed assembly but they only contain data ('.long' directive), which makes them portable accross CPUs. +# Only the endianness and the character set (ASCII, EBCDIC) makes a difference, also see +# https://unicode-org.github.io/icu/userguide/icu_data/#sharing-icu-data-between-platforms, 'Sharing ICU Data Between Platforms') +# (and as an experiment, try re-generating the data files on x86 vs. ARM, ... you'll get exactly the same files) set(ICUDATA_SOURCES "${ICUDATA_SOURCE_FILE}" diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 3df9655701c..a39f96867be 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -187,10 +187,15 @@ function setup_logs_replication ') echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2 + echo "::group::${table}" # there's the only way big "$statement" can be printed without causing EAGAIN error # cat: write error: Resource temporarily unavailable - echo "$statement" | cat + statement_print="${statement}" + if [ "${#statement_print}" -gt 4000 ]; then + statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}" + fi + echo -e "$statement_print" echo "::endgroup::" echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ diff --git a/docs/en/engines/table-engines/integrations/azure-queue.md b/docs/en/engines/table-engines/integrations/azure-queue.md new file mode 100644 index 00000000000..b5259336a8b --- /dev/null +++ b/docs/en/engines/table-engines/integrations/azure-queue.md @@ -0,0 +1,72 @@ +--- +slug: /en/engines/table-engines/integrations/azure-queue +sidebar_position: 181 +sidebar_label: AzureQueue +--- + +# AzureQueue Table Engine + +This engine provides an integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem, allowing streaming data import. + +## Create Table {#creating-a-table} + +``` sql +CREATE TABLE test (name String, value UInt32) + ENGINE = AzureQueue(...) + [SETTINGS] + [mode = '',] + [after_processing = 'keep',] + [keeper_path = '',] + ... +``` + +**Engine parameters** + +`AzureQueue` parameters are the same as `AzureBlobStorage` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/azureBlobStorage.md). + +**Example** + +```sql +CREATE TABLE azure_queue_engine_table (name String, value UInt32) +ENGINE=AzureQueue('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/data/') +SETTINGS + mode = 'unordered' +``` + +## Settings {#settings} + +The set of supported settings is the same as for `S3Queue` table engine, but without `s3queue_` prefix. See [full list of settings settings](../../../engines/table-engines/integrations/s3queue.md#settings). + +## Description {#description} + +`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: + +1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream. +2. Create a table with the desired structure. +3. Create a materialized view that converts data from the engine and puts it into a previously created table. + +When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. + +Example: + +``` sql + CREATE TABLE azure_queue_engine_table (name String, value UInt32) + ENGINE=AzureQueue('', 'CSV', 'gzip') + SETTINGS + mode = 'unordered'; + + CREATE TABLE stats (name String, value UInt32) + ENGINE = MergeTree() ORDER BY name; + + CREATE MATERIALIZED VIEW consumer TO stats + AS SELECT name, value FROM azure_queue_engine_table; + + SELECT * FROM stats ORDER BY name; +``` + +## Virtual columns {#virtual-columns} + +- `_path` — Path to the file. +- `_file` — Name of the file. + +For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 48a08dfa499..f02d0563491 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -35,7 +35,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) [SETTINGS ...] ``` -### Engine parameters +### Engine parameters {#parameters} - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 06325fa15fb..f1957cfe1ce 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -5,6 +5,7 @@ sidebar_label: S3Queue --- # S3Queue Table Engine + This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features. ## Create Table {#creating-a-table} @@ -16,27 +17,25 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [mode = '',] [after_processing = 'keep',] [keeper_path = '',] - [s3queue_loading_retries = 0,] - [s3queue_processing_threads_num = 1,] - [s3queue_enable_logging_to_s3queue_log = 0,] - [s3queue_polling_min_timeout_ms = 1000,] - [s3queue_polling_max_timeout_ms = 10000,] - [s3queue_polling_backoff_ms = 0,] - [s3queue_tracked_file_ttl_sec = 0,] - [s3queue_tracked_files_limit = 1000,] - [s3queue_cleanup_interval_min_ms = 10000,] - [s3queue_cleanup_interval_max_ms = 30000,] + [loading_retries = 0,] + [processing_threads_num = 1,] + [enable_logging_to_s3queue_log = 0,] + [polling_min_timeout_ms = 1000,] + [polling_max_timeout_ms = 10000,] + [polling_backoff_ms = 0,] + [tracked_file_ttl_sec = 0,] + [tracked_files_limit = 1000,] + [cleanup_interval_min_ms = 10000,] + [cleanup_interval_max_ms = 30000,] ``` -Starting with `24.7` settings without `s3queue_` prefix are also supported. +:::warning +Before `24.7`, it is required to use `s3queue_` prefix for all settings apart from `mode`, `after_processing` and `keeper_path`. +::: **Engine parameters** -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). -- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. -- `format` — The [format](../../../interfaces/formats.md#formats) of the file. -- `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. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +`S3Queue` parameters are the same as `S3` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/s3.md#parameters). **Example** diff --git a/docs/en/operations/external-authenticators/ssl-x509.md b/docs/en/operations/external-authenticators/ssl-x509.md index 09fac45d7ae..a7514966fa7 100644 --- a/docs/en/operations/external-authenticators/ssl-x509.md +++ b/docs/en/operations/external-authenticators/ssl-x509.md @@ -6,7 +6,7 @@ import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.m -[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration. +[SSL 'strict' option](../server-configuration-parameters/settings.md#server_configuration_parameters-openssl) enables mandatory certificate validation for the incoming connections. In this case, only connections with trusted certificates can be established. Connections with untrusted certificates will be rejected. Thus, certificate validation allows to uniquely authenticate an incoming connection. `Common Name` or `subjectAltName extension` field of the certificate is used to identify the connected user. `subjectAltName extension` supports the usage of one wildcard '*' in the server configuration. This allows to associate multiple certificates with the same user. Additionally, reissuing and revoking of the certificates does not affect the ClickHouse configuration. To enable SSL certificate authentication, a list of `Common Name`'s or `Subject Alt Name`'s for each ClickHouse user must be specified in the settings file `users.xml `: @@ -30,6 +30,12 @@ To enable SSL certificate authentication, a list of `Common Name`'s or `Subject + + + + URI:spiffe://foo.com/*/bar + + ``` diff --git a/docs/en/operations/utilities/clickhouse-keeper-client.md b/docs/en/operations/utilities/clickhouse-keeper-client.md index fbfdd66d1a3..6f026766750 100644 --- a/docs/en/operations/utilities/clickhouse-keeper-client.md +++ b/docs/en/operations/utilities/clickhouse-keeper-client.md @@ -55,7 +55,7 @@ keeper foo bar - `touch ''` -- Creates new node with an empty string as value. Doesn't throw an exception if the node already exists - `get ''` -- Returns the node's value - `rm '' [version]` -- Removes the node only if version matches (default: -1) -- `rmr ''` -- Recursively deletes path. Confirmation required +- `rmr '' [limit]` -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100) - `flwc ` -- Executes four-letter-word command - `help` -- Prints this message - `get_direct_children_number '[path]'` -- Get numbers of direct children nodes under a specific path diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 4ad2eb31e6d..b4a5329b01f 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -506,14 +506,23 @@ bool RMRCommand::parse(IParser::Pos & pos, std::shared_ptr & nod return false; node->args.push_back(std::move(path)); + ASTPtr remove_nodes_limit; + if (ParserUnsignedInteger{}.parse(pos, remove_nodes_limit, expected)) + node->args.push_back(remove_nodes_limit->as().value); + else + node->args.push_back(UInt64(100)); + return true; } void RMRCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const { String path = client->getAbsolutePath(query->args[0].safeGet()); + UInt64 remove_nodes_limit = query->args[1].safeGet(); + client->askConfirmation( - "You are going to recursively delete path " + path, [client, path] { client->zookeeper->removeRecursive(path); }); + "You are going to recursively delete path " + path, + [client, path, remove_nodes_limit] { client->zookeeper->removeRecursive(path, static_cast(remove_nodes_limit)); }); } bool ReconfigCommand::parse(IParser::Pos & pos, std::shared_ptr & node, DB::Expected & expected) const diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index 686a752b6b6..da577ce1e65 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -184,7 +184,7 @@ class RMRCommand : public IKeeperClientCommand void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; - String getHelpMessage() const override { return "{} -- Recursively deletes path. Confirmation required"; } + String getHelpMessage() const override { return "{} [limit] -- Recursively deletes path if the subtree size is smaller than the limit. Confirmation required (default limit = 100)"; } }; class ReconfigCommand : public IKeeperClientCommand diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index ced661d9772..3007df60765 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -384,6 +385,9 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 0)); + memory_worker.start(); + static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -425,8 +429,9 @@ try for (const auto & server : *servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); @@ -655,7 +660,6 @@ try GWPAsan::initFinished(); #endif - LOG_INFO(log, "Ready for connections."); waitForTerminationRequest(); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index fb5717ba33f..c69d822e383 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -25,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -111,6 +111,8 @@ #include #include +#include + #include "config.h" #include @@ -449,9 +451,12 @@ void checkForUsersNotInMainConfig( } } +namespace +{ + /// Unused in other builds #if defined(OS_LINUX) -static String readLine(const String & path) +String readLine(const String & path) { ReadBufferFromFile in(path); String contents; @@ -459,7 +464,7 @@ static String readLine(const String & path) return contents; } -static int readNumber(const String & path) +int readNumber(const String & path) { ReadBufferFromFile in(path); int result; @@ -469,7 +474,7 @@ static int readNumber(const String & path) #endif -static void sanityChecks(Server & server) +void sanityChecks(Server & server) { std::string data_path = getCanonicalPath(server.config().getString("path", DBMS_DEFAULT_PATH)); std::string logs_path = server.config().getString("logger.log", ""); @@ -590,6 +595,8 @@ static void sanityChecks(Server & server) } } +} + void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) { try @@ -906,6 +913,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -924,8 +933,9 @@ try for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads(), server.refusedConnections()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); /// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown() /// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads. @@ -1204,6 +1214,8 @@ try FailPointInjection::enableFromGlobalConfig(config()); + memory_worker.start(); + int default_oom_score = 0; #if !defined(NDEBUG) @@ -1547,15 +1559,6 @@ try total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - if (cgroups_memory_usage_observer) - { - double hard_limit_ratio = new_server_settings.cgroup_memory_watcher_hard_limit_ratio; - double soft_limit_ratio = new_server_settings.cgroup_memory_watcher_soft_limit_ratio; - cgroups_memory_usage_observer->setMemoryUsageLimits( - static_cast(max_server_memory_usage * hard_limit_ratio), - static_cast(max_server_memory_usage * soft_limit_ratio)); - } - size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit; size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio); @@ -1584,8 +1587,6 @@ try background_memory_tracker.setDescription("(background)"); background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking); - total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory); - auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 6b9a6e05cf6..90c82b466db 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -239,15 +239,36 @@ bool Authentication::areCredentialsValid( throw Authentication::Require(auth_data.getKerberosRealm()); case AuthenticationType::SSL_CERTIFICATE: + { for (SSLCertificateSubjects::Type type : {SSLCertificateSubjects::Type::CN, SSLCertificateSubjects::Type::SAN}) { for (const auto & subject : auth_data.getSSLCertificateSubjects().at(type)) { if (ssl_certificate_credentials->getSSLCertificateSubjects().at(type).contains(subject)) return true; + + // Wildcard support (1 only) + if (subject.contains('*')) + { + auto prefix = std::string_view(subject).substr(0, subject.find('*')); + auto suffix = std::string_view(subject).substr(subject.find('*') + 1); + auto slashes = std::count(subject.begin(), subject.end(), '/'); + + for (const auto & certificate_subject : ssl_certificate_credentials->getSSLCertificateSubjects().at(type)) + { + bool matches_wildcard = certificate_subject.starts_with(prefix) && certificate_subject.ends_with(suffix); + + // '*' must not represent a '/' in URI, so check if the number of '/' are equal + bool matches_slashes = slashes == count(certificate_subject.begin(), certificate_subject.end(), '/'); + + if (matches_wildcard && matches_slashes) + return true; + } + } } } return false; + } case AuthenticationType::SSH_KEY: #if USE_SSH diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 636ac80e350..8fb0b645096 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -116,15 +116,17 @@ class GroupConcatImpl final SerializationPtr serialization; UInt64 limit; const String delimiter; + const DataTypePtr type; public: GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_) : IAggregateFunctionDataHelper, GroupConcatImpl>( {data_type_}, parameters_, std::make_shared()) - , serialization(this->argument_types[0]->getDefaultSerialization()) , limit(limit_) , delimiter(delimiter_) + , type(data_type_) { + serialization = isFixedString(type) ? std::make_shared()->getDefaultSerialization() : this->argument_types[0]->getDefaultSerialization(); } String getName() const override { return name; } @@ -140,7 +142,14 @@ public: if (cur_data.data_size != 0) cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); - cur_data.insert(columns[0], serialization, row_num, arena); + if (isFixedString(type)) + { + ColumnWithTypeAndName col = {columns[0]->getPtr(), type, "column"}; + const auto & col_str = castColumn(col, std::make_shared()); + cur_data.insert(col_str.get(), serialization, row_num, arena); + } + else + cur_data.insert(columns[0], serialization, row_num, arena); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1889bba3b39..f0410eee9fe 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -176,7 +176,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp) target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 9b6a7428411..a92d321f8aa 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1,5 +1,3 @@ -#include - #include #include #include @@ -8,8 +6,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -69,10 +69,14 @@ static void openCgroupv2MetricFile(const std::string & filename, std::optional(jemalloc_full_name.c_str()); values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html"); return value; } @@ -768,8 +770,11 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. - auto epoch = updateJemallocEpoch(); - new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; + auto epoch = update_jemalloc_epoch ? updateJemallocEpoch() : getJemallocValue("epoch"); + new_values["jemalloc.epoch"] + = {epoch, + "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other " + "`jemalloc` metrics."}; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); @@ -782,10 +787,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) saveJemallocMetric(new_values, "background_thread.num_threads"); saveJemallocMetric(new_values, "background_thread.num_runs"); saveJemallocMetric(new_values, "background_thread.run_intervals"); - saveJemallocProf(new_values, "active"); + saveJemallocProf(new_values, "active"); saveAllArenasMetric(new_values, "pactive"); - [[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric(new_values, "pdirty"); - [[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric(new_values, "pmuzzy"); + saveAllArenasMetric(new_values, "pdirty"); + saveAllArenasMetric(new_values, "pmuzzy"); saveAllArenasMetric(new_values, "dirty_purged"); saveAllArenasMetric(new_values, "muzzy_purged"); #endif @@ -814,41 +819,8 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; - /// We must update the value of total_memory_tracker periodically. - /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. - /// See https://github.com/ClickHouse/ClickHouse/issues/10293 - { - Int64 amount = total_memory_tracker.get(); - Int64 peak = total_memory_tracker.getPeak(); - Int64 rss = data.resident; - Int64 free_memory_in_allocator_arenas = 0; - -#if USE_JEMALLOC - /// According to jemalloc man, pdirty is: - /// - /// Number of pages within unused extents that are potentially - /// dirty, and for which madvise() or similar has not been called. - /// - /// So they will be subtracted from RSS to make accounting more - /// accurate, since those pages are not really RSS but a memory - /// that can be used at anytime via jemalloc. - free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); -#endif - - Int64 difference = rss - amount; - - /// Log only if difference is high. This is for convenience. The threshold is arbitrary. - if (difference >= 1048576 || difference <= -1048576) - LOG_TRACE(log, - "MemoryTracking: was {}, peak {}, free memory in arenas {}, will set to {} (RSS), difference: {}", - ReadableSize(amount), - ReadableSize(peak), - ReadableSize(free_memory_in_allocator_arenas), - ReadableSize(rss), - ReadableSize(difference)); - - MemoryTracker::setRSS(rss, free_memory_in_allocator_arenas); - } + if (update_rss) + MemoryTracker::updateRSS(data.resident); } { diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 78d07ef4b6c..215dc6e1337 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -1,15 +1,14 @@ #pragma once +#include #include #include #include #include #include -#include #include #include -#include #include #include #include @@ -69,7 +68,9 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); virtual ~AsynchronousMetrics(); @@ -112,6 +113,9 @@ private: MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif + [[maybe_unused]] const bool update_jemalloc_epoch; + [[maybe_unused]] const bool update_rss; + #if defined(OS_LINUX) std::optional meminfo TSA_GUARDED_BY(data_mutex); std::optional loadavg TSA_GUARDED_BY(data_mutex); diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 83b04360164..28bb861865a 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -14,239 +14,21 @@ #include #include -#include -#include -#include - -#include "config.h" -#if USE_JEMALLOC -# include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) -#endif using namespace DB; -namespace fs = std::filesystem; - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int FILE_DOESNT_EXIST; -extern const int INCORRECT_DATA; -} - -} - -namespace -{ - -/// Format is -/// kernel 5 -/// rss 15 -/// [...] -using Metrics = std::map; - -Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf) -{ - Metrics metrics; - while (!buf.eof()) - { - std::string current_key; - readStringUntilWhitespace(current_key, buf); - - assertChar(' ', buf); - - uint64_t value = 0; - readIntText(value, buf); - assertChar('\n', buf); - - auto [_, inserted] = metrics.emplace(std::move(current_key), value); - chassert(inserted, "Duplicate keys in stat file"); - } - return metrics; -} - -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) -{ - const auto all_metrics = readAllMetricsFromStatFile(buf); - if (const auto it = all_metrics.find(key); it != all_metrics.end()) - return it->second; - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); -} - -struct CgroupsV1Reader : ICgroupsReader -{ - explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - buf.rewind(); - return readMetricFromStatFile(buf, "rss"); - } - - std::string dumpAllStats() override - { - std::lock_guard lock(mutex); - buf.rewind(); - return fmt::format("{}", readAllMetricsFromStatFile(buf)); - } - -private: - std::mutex mutex; - ReadBufferFromFile buf TSA_GUARDED_BY(mutex); -}; - -struct CgroupsV2Reader : ICgroupsReader -{ - explicit CgroupsV2Reader(const fs::path & stat_file_dir) - : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") - { - } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - current_buf.rewind(); - stat_buf.rewind(); - - int64_t mem_usage = 0; - /// memory.current contains a single number - /// the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 - readIntText(mem_usage, current_buf); - mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file"); - chassert(mem_usage >= 0, "Negative memory usage"); - return mem_usage; - } - - std::string dumpAllStats() override - { - std::lock_guard lock(mutex); - stat_buf.rewind(); - return fmt::format("{}", readAllMetricsFromStatFile(stat_buf)); - } - -private: - std::mutex mutex; - ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex); - ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); -}; - -/// Caveats: -/// - All of the logic in this file assumes that the current process is the only process in the -/// containing cgroup (or more precisely: the only process with significant memory consumption). -/// If this is not the case, then other processe's memory consumption may affect the internal -/// memory tracker ... -/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a -/// decade and will go away at some point, hierarchical detection is only implemented for v2. -/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such -/// systems existed only for a short transition period. - -std::optional getCgroupsV1Path() -{ - auto path = default_cgroups_mount / "memory/memory.stat"; - if (!fs::exists(path)) - return {}; - return {default_cgroups_mount / "memory"}; -} - -std::pair getCgroupsPath() -{ - auto v2_path = getCgroupsV2PathContainingFile("memory.current"); - if (v2_path.has_value()) - return {*v2_path, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; - - auto v1_path = getCgroupsV1Path(); - if (v1_path.has_value()) - return {*v1_path, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; - - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); -} - -} namespace DB { CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) -{ - const auto [cgroup_path, version] = getCgroupsPath(); - - cgroup_reader = createCgroupsReader(version, cgroup_path); - - LOG_INFO( - log, - "Will read the current memory usage from '{}' (cgroups version: {}), wait time is {} sec", - cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2", - wait_time.count()); -} +{} CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() { stopThread(); } -void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_) -{ - std::lock_guard limit_lock(limit_mutex); - - if (hard_limit_ == hard_limit && soft_limit_ == soft_limit) - return; - - hard_limit = hard_limit_; - soft_limit = soft_limit_; - - on_hard_limit = [this, hard_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_)); - - /// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are - /// really zero. Trying to avoid OOM ... - MemoryTracker::setRSS(hard_limit_, 0); - } - else - { - LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_)); - } - }; - - on_soft_limit = [this, soft_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); - -# if USE_JEMALLOC - LOG_INFO(log, "Purging jemalloc arenas"); - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); -# endif - /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE( - log, - "Read current memory usage {} bytes ({}) from cgroups, full available stats: {}", - memory_usage, - ReadableSize(memory_usage), - cgroup_reader->dumpAllStats()); - MemoryTracker::setRSS(memory_usage, 0); - - LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); - } - else - { - LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_)); - } - }; - - LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_)); -} - void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_) { std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); @@ -300,35 +82,6 @@ void CgroupsMemoryUsageObserver::runThread() std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); on_memory_amount_available_changed(); } - - std::lock_guard limit_lock(limit_mutex); - if (soft_limit > 0 && hard_limit > 0) - { - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); - if (memory_usage > hard_limit) - { - if (last_memory_usage <= hard_limit) - on_hard_limit(true); - } - else - { - if (last_memory_usage > hard_limit) - on_hard_limit(false); - } - - if (memory_usage > soft_limit) - { - if (last_memory_usage <= soft_limit) - on_soft_limit(true); - } - else - { - if (last_memory_usage > soft_limit) - on_soft_limit(false); - } - last_memory_usage = memory_usage; - } } catch (...) { @@ -337,13 +90,6 @@ void CgroupsMemoryUsageObserver::runThread() } } -std::unique_ptr createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const fs::path & cgroup_path) -{ - if (version == CgroupsMemoryUsageObserver::CgroupsVersion::V2) - return std::make_unique(cgroup_path); - else - return std::make_unique(cgroup_path); -} } #endif diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 7f888fe631b..3de83d6b437 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -3,53 +3,27 @@ #include #include -#include #include namespace DB { -struct ICgroupsReader -{ - virtual ~ICgroupsReader() = default; - - virtual uint64_t readMemoryUsage() = 0; - - virtual std::string dumpAllStats() = 0; -}; - -/// Does two things: -/// 1. Periodically reads the memory usage of the process from Linux cgroups. -/// You can specify soft or hard memory limits: -/// - When the soft memory limit is hit, drop jemalloc cache. -/// - When the hard memory limit is hit, update MemoryTracking metric to throw memory exceptions faster. -/// The goal of this is to avoid that the process hits the maximum allowed memory limit at which there is a good -/// chance that the Limux OOM killer terminates it. All of this is done is because internal memory tracking in -/// ClickHouse can unfortunately under-estimate the actually used memory. -/// 2. Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). -/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server -/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' -/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). -/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling -/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes -/// to the database. +/// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). +/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server +/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' +/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). +/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling +/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes +/// to the database. #if defined(OS_LINUX) class CgroupsMemoryUsageObserver { public: - using OnMemoryLimitFn = std::function; using OnMemoryAmountAvailableChangedFn = std::function; - enum class CgroupsVersion : uint8_t - { - V1, - V2 - }; - explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_); ~CgroupsMemoryUsageObserver(); - void setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_); void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_); void startThread(); @@ -60,32 +34,22 @@ private: const std::chrono::seconds wait_time; std::mutex limit_mutex; - size_t hard_limit TSA_GUARDED_BY(limit_mutex) = 0; - size_t soft_limit TSA_GUARDED_BY(limit_mutex) = 0; - OnMemoryLimitFn on_hard_limit TSA_GUARDED_BY(limit_mutex); - OnMemoryLimitFn on_soft_limit TSA_GUARDED_BY(limit_mutex); std::mutex memory_amount_available_changed_mutex; OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex); - uint64_t last_memory_usage = 0; /// how much memory does the process use uint64_t last_available_memory_amount; /// how much memory can the process use void stopThread(); void runThread(); - std::unique_ptr cgroup_reader; - std::mutex thread_mutex; std::condition_variable cond; ThreadFromGlobalPool thread; bool quit = false; }; -std::unique_ptr -createCgroupsReader(CgroupsMemoryUsageObserver::CgroupsVersion version, const std::filesystem::path & cgroup_path); - #else class CgroupsMemoryUsageObserver { diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index d7cc246db6a..d8ff9268cca 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #define STRINGIFY_HELPER(x) #x #define STRINGIFY(x) STRINGIFY_HELPER(x) @@ -26,7 +25,6 @@ namespace ErrorCodes void purgeJemallocArenas() { - LOG_TRACE(getLogger("SystemJemalloc"), "Purging unused memory"); Stopwatch watch; mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); @@ -46,20 +44,6 @@ void checkJemallocProfilingEnabled() "set: MALLOC_CONF=background_thread:true,prof:true"); } -template -void setJemallocValue(const char * name, T value) -{ - T old_value; - size_t old_value_size = sizeof(T); - if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return; - } - - LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); -} - void setJemallocProfileActive(bool value) { checkJemallocProfilingEnabled(); diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 499a906fd3d..22a94a44eba 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -5,6 +5,8 @@ #if USE_JEMALLOC #include +#include +#include namespace DB { @@ -21,6 +23,59 @@ void setJemallocBackgroundThreads(bool enabled); void setJemallocMaxBackgroundThreads(size_t max_threads); +template +void setJemallocValue(const char * name, T value) +{ + T old_value; + size_t old_value_size = sizeof(T); + mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T)); + LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); +} + +template +T getJemallocValue(const char * name) +{ + T value; + size_t value_size = sizeof(T); + mallctl(name, &value, &value_size, nullptr, 0); + return value; +} + +/// Each mallctl call consists of string name lookup which can be expensive. +/// This can be avoided by translating name to "Management Information Base" (MIB) +/// and using it in mallctlbymib calls +template +struct JemallocMibCache +{ + explicit JemallocMibCache(const char * name) + { + mallctlnametomib(name, mib, &mib_length); + } + + void setValue(T value) + { + mallctlbymib(mib, mib_length, nullptr, nullptr, reinterpret_cast(&value), sizeof(T)); + } + + T getValue() + { + T value; + size_t value_size = sizeof(T); + mallctlbymib(mib, mib_length, &value, &value_size, nullptr, 0); + return value; + } + + void run() + { + mallctlbymib(mib, mib_length, nullptr, nullptr, nullptr, 0); + } + +private: + static constexpr size_t max_mib_length = 4; + size_t mib[max_mib_length]; + size_t mib_length = max_mib_length; +}; + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 7c0115467c6..7bf665ea7a0 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -20,13 +20,9 @@ #if USE_JEMALLOC # include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) - #endif #include -#include #include #include #include @@ -115,8 +111,6 @@ void AllocationTrace::onFreeImpl(void * ptr, size_t size) const namespace ProfileEvents { extern const Event QueryMemoryLimitExceeded; - extern const Event MemoryAllocatorPurge; - extern const Event MemoryAllocatorPurgeTimeMicroseconds; } using namespace std::chrono_literals; @@ -126,15 +120,13 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false); -std::atomic MemoryTracker::free_memory_in_allocator_arenas; - MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} + MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_) - : parent(parent_) - , log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_) - , level(level_) -{} + : parent(parent_), log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_), level(level_) +{ +} MemoryTracker::~MemoryTracker() { @@ -204,10 +196,14 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] return; MemoryTrackerBlockerInThread blocker(VariableContext::Global); - LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " - "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); + LOG_TEST( + getLogger("MemoryTracker"), + "Too big allocation ({} bytes) without checking memory limits, " + "it may lead to OOM. Stack trace: {}", + size, + StackTrace().toString()); #else - return; /// Avoid trash logging in release builds + /// Avoid trash logging in release builds #endif } @@ -228,6 +224,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// For global memory tracker always update memory usage. amount.fetch_add(size, std::memory_order_relaxed); + rss.fetch_add(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) @@ -249,6 +246,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed * So, we allow over-allocations. */ Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); + Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end() && size) @@ -275,6 +273,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -297,33 +296,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - Int64 limit_to_check = current_hard_limit; - -#if USE_JEMALLOC - if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed)) - { - /// Jemalloc arenas may keep some extra memory. - /// This memory was substucted from RSS to decrease memory drift. - /// In case memory is close to limit, try to pugre the arenas. - /// This is needed to avoid OOM, because some allocations are directly done with mmap. - Int64 current_free_memory_in_allocator_arenas = free_memory_in_allocator_arenas.load(std::memory_order_relaxed); - - if (current_free_memory_in_allocator_arenas > 0 && current_hard_limit && current_free_memory_in_allocator_arenas + will_be > current_hard_limit) - { - if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0) - { - Stopwatch watch; - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); - } - } - - limit_to_check += abs(current_free_memory_in_allocator_arenas); - } -#endif - - if (unlikely(current_hard_limit && will_be > limit_to_check)) + if (unlikely( + current_hard_limit && (will_be > current_hard_limit || (level == VariableContext::Global && will_be_rss > current_hard_limit)))) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -335,6 +309,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -343,12 +318,13 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, "Memory limit{}{} exceeded: " - "would use {} (attempt to allocate chunk of {} bytes), maximum: {}." + "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." "{}{}", description ? " " : "", description ? description : "", formatReadableSizeWithBinarySuffix(will_be), size, + formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), formatReadableSizeWithBinarySuffix(current_hard_limit), overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ", toDescription(overcommit_result)); @@ -442,6 +418,7 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) { /// For global memory tracker always update memory usage. amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::sub(metric_loaded, size); @@ -455,7 +432,12 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) } Int64 accounted_size = size; - if (level == VariableContext::Thread || level == VariableContext::Global) + if (level == VariableContext::Global) + { + amount.fetch_sub(accounted_size, std::memory_order_relaxed); + rss.fetch_sub(accounted_size, std::memory_order_relaxed); + } + else if (level == VariableContext::Thread) { /// Could become negative if memory allocated in this thread is freed in another one amount.fetch_sub(accounted_size, std::memory_order_relaxed); @@ -529,21 +511,29 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_) +void MemoryTracker::updateRSS(Int64 rss_) { - Int64 new_amount = rss_; + total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); +} + +void MemoryTracker::updateAllocated(Int64 allocated_) +{ + Int64 new_amount = allocated_; + LOG_INFO( + getLogger("MemoryTracker"), + "Correcting the value of global memory tracker from {} to {}", + ReadableSize(total_memory_tracker.amount.load(std::memory_order_relaxed)), + ReadableSize(allocated_)); total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); - free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::set(metric_loaded, new_amount); bool log_memory_usage = true; - total_memory_tracker.updatePeak(rss_, log_memory_usage); + total_memory_tracker.updatePeak(new_amount, log_memory_usage); } - void MemoryTracker::setSoftLimit(Int64 value) { soft_limit.store(value, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index fd32b631774..f15465a20c1 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -57,9 +56,8 @@ private: std::atomic soft_limit {0}; std::atomic hard_limit {0}; std::atomic profiler_limit {0}; - std::atomic_bool allow_use_jemalloc_memory {true}; - static std::atomic free_memory_in_allocator_arenas; + std::atomic rss{0}; Int64 profiler_step = 0; @@ -122,6 +120,11 @@ public: return amount.load(std::memory_order_relaxed); } + Int64 getRSS() const + { + return rss.load(std::memory_order_relaxed); + } + // Merges and mutations may pass memory ownership to other threads thus in the end of execution // MemoryTracker for background task may have a non-zero counter. // This method is intended to fix the counter inside of background_memory_tracker. @@ -154,14 +157,6 @@ public: { return soft_limit.load(std::memory_order_relaxed); } - void setAllowUseJemallocMemory(bool value) - { - allow_use_jemalloc_memory.store(value, std::memory_order_relaxed); - } - bool getAllowUseJemallocMmemory() const - { - return allow_use_jemalloc_memory.load(std::memory_order_relaxed); - } /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. @@ -249,10 +244,9 @@ public: /// Reset the accumulated data. void reset(); - /// Reset current counter to an RSS value. - /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. - /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_); + /// update values based on external information (e.g. jemalloc's stat) + static void updateRSS(Int64 rss_); + static void updateAllocated(Int64 allocated_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp new file mode 100644 index 00000000000..11f3bff348c --- /dev/null +++ b/src/Common/MemoryWorker.cpp @@ -0,0 +1,333 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include + +namespace fs = std::filesystem; + +namespace ProfileEvents +{ + extern const Event MemoryAllocatorPurge; + extern const Event MemoryAllocatorPurgeTimeMicroseconds; + extern const Event MemoryWorkerRun; + extern const Event MemoryWorkerRunElapsedMicroseconds; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int FILE_DOESNT_EXIST; + extern const int LOGICAL_ERROR; +} + +#if defined(OS_LINUX) +namespace +{ + +using Metrics = std::map; + +/// Format is +/// kernel 5 +/// rss 15 +/// [...] +Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf) +{ + Metrics metrics; + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + + assertChar(' ', buf); + + uint64_t value = 0; + readIntText(value, buf); + assertChar('\n', buf); + + auto [_, inserted] = metrics.emplace(std::move(current_key), value); + chassert(inserted, "Duplicate keys in stat file"); + } + return metrics; +} + +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, std::string_view key) +{ + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t value = 0; + readIntText(value, buf); + return value; + } + LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName()); + return 0; +} + +struct CgroupsV1Reader : ICgroupsReader +{ + explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return readMetricFromStatFile(buf, "rss"); + } + + std::string dumpAllStats() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return fmt::format("{}", readAllMetricsFromStatFile(buf)); + } + +private: + std::mutex mutex; + ReadBufferFromFile buf TSA_GUARDED_BY(mutex); +}; + +struct CgroupsV2Reader : ICgroupsReader +{ + explicit CgroupsV2Reader(const fs::path & stat_file_dir) : stat_buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + stat_buf.rewind(); + return readMetricFromStatFile(stat_buf, "anon"); + } + + std::string dumpAllStats() override + { + std::lock_guard lock(mutex); + stat_buf.rewind(); + return fmt::format("{}", readAllMetricsFromStatFile(stat_buf)); + } + +private: + std::mutex mutex; + ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); +}; + +/// Caveats: +/// - All of the logic in this file assumes that the current process is the only process in the +/// containing cgroup (or more precisely: the only process with significant memory consumption). +/// If this is not the case, then other processe's memory consumption may affect the internal +/// memory tracker ... +/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a +/// decade and will go away at some point, hierarchical detection is only implemented for v2. +/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such +/// systems existed only for a short transition period. + +std::optional getCgroupsV1Path() +{ + auto path = default_cgroups_mount / "memory/memory.stat"; + if (!fs::exists(path)) + return {}; + return {default_cgroups_mount / "memory"}; +} + +std::pair getCgroupsPath() +{ + auto v2_path = getCgroupsV2PathContainingFile("memory.current"); + if (v2_path.has_value()) + return {*v2_path, ICgroupsReader::CgroupsVersion::V2}; + + auto v1_path = getCgroupsV1Path(); + if (v1_path.has_value()) + return {*v1_path, ICgroupsReader::CgroupsVersion::V1}; + + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); +} + +} + +std::shared_ptr ICgroupsReader::createCgroupsReader(ICgroupsReader::CgroupsVersion version, const std::filesystem::path & cgroup_path) +{ + if (version == CgroupsVersion::V2) + return std::make_shared(cgroup_path); + else + { + chassert(version == CgroupsVersion::V1); + return std::make_shared(cgroup_path); + } +} +#endif + +namespace +{ + +std::string_view sourceToString(MemoryWorker::MemoryUsageSource source) +{ + switch (source) + { + case MemoryWorker::MemoryUsageSource::Cgroups: return "Cgroups"; + case MemoryWorker::MemoryUsageSource::Jemalloc: return "Jemalloc"; + case MemoryWorker::MemoryUsageSource::None: return "None"; + } +} + +} + +/// We try to pick the best possible supported source for reading memory usage. +/// Supported sources in order of priority +/// - reading from cgroups' pseudo-files (fastest and most accurate) +/// - reading jemalloc's resident stat (doesn't take into account allocations that didn't use jemalloc) +/// Also, different tick rates are used because not all options are equally fast +MemoryWorker::MemoryWorker(uint64_t period_ms_) + : log(getLogger("MemoryWorker")) + , period_ms(period_ms_) +{ +#if defined(OS_LINUX) + try + { + static constexpr uint64_t cgroups_memory_usage_tick_ms{50}; + + const auto [cgroup_path, version] = getCgroupsPath(); + LOG_INFO( + getLogger("CgroupsReader"), + "Will create cgroup reader from '{}' (cgroups version: {})", + cgroup_path, + (version == ICgroupsReader::CgroupsVersion::V1) ? "v1" : "v2"); + + cgroups_reader = ICgroupsReader::createCgroupsReader(version, cgroup_path); + source = MemoryUsageSource::Cgroups; + if (period_ms == 0) + period_ms = cgroups_memory_usage_tick_ms; + + return; + } + catch (...) + { + tryLogCurrentException(log, "Cannot use cgroups reader"); + } +#endif + +#if USE_JEMALLOC + static constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; + + source = MemoryUsageSource::Jemalloc; + if (period_ms == 0) + period_ms = jemalloc_memory_usage_tick_ms; +#endif +} + +MemoryWorker::MemoryUsageSource MemoryWorker::getSource() +{ + return source; +} + +void MemoryWorker::start() +{ + if (source == MemoryUsageSource::None) + return; + + LOG_INFO( + getLogger("MemoryWorker"), + "Starting background memory thread with period of {}ms, using {} as source", + period_ms, + sourceToString(source)); + background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); +} + +MemoryWorker::~MemoryWorker() +{ + { + std::unique_lock lock(mutex); + shutdown = true; + } + cv.notify_all(); + + if (background_thread.joinable()) + background_thread.join(); +} + +uint64_t MemoryWorker::getMemoryUsage() +{ + switch (source) + { + case MemoryUsageSource::Cgroups: + return cgroups_reader != nullptr ? cgroups_reader->readMemoryUsage() : 0; + case MemoryUsageSource::Jemalloc: +#if USE_JEMALLOC + return resident_mib.getValue(); +#else + return 0; +#endif + case MemoryUsageSource::None: + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used"); + } +} + +void MemoryWorker::backgroundThread() +{ + std::chrono::milliseconds chrono_period_ms{period_ms}; + [[maybe_unused]] bool first_run = true; + std::unique_lock lock(mutex); + while (true) + { + cv.wait_for(lock, chrono_period_ms, [this] { return shutdown; }); + if (shutdown) + return; + + Stopwatch total_watch; + +#if USE_JEMALLOC + if (source == MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); +#endif + + Int64 resident = getMemoryUsage(); + MemoryTracker::updateRSS(resident); + +#if USE_JEMALLOC + if (resident > total_memory_tracker.getHardLimit()) + { + Stopwatch purge_watch; + purge_mib.run(); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); + } +#endif + +#if USE_JEMALLOC + if (unlikely(first_run || total_memory_tracker.get() < 0)) + { + if (source != MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); + + MemoryTracker::updateAllocated(allocated_mib.getValue()); + } +#endif + + ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); + ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); + first_run = false; + } +} + +} diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h new file mode 100644 index 00000000000..f4b0fed23ec --- /dev/null +++ b/src/Common/MemoryWorker.h @@ -0,0 +1,84 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct ICgroupsReader +{ + enum class CgroupsVersion : uint8_t + { + V1, + V2 + }; + +#if defined(OS_LINUX) + static std::shared_ptr + createCgroupsReader(ICgroupsReader::CgroupsVersion version, const std::filesystem::path & cgroup_path); +#endif + + virtual ~ICgroupsReader() = default; + + virtual uint64_t readMemoryUsage() = 0; + + virtual std::string dumpAllStats() = 0; +}; + + +/// Correct MemoryTracker based on external information (e.g. Cgroups or stats.resident from jemalloc) +/// The worker spawns a background thread which periodically reads current resident memory from the source, +/// whose value is sent to global MemoryTracker. +/// It can do additional things like purging jemalloc dirty pages if the current memory usage is higher than global hard limit. +class MemoryWorker +{ +public: + explicit MemoryWorker(uint64_t period_ms_); + + enum class MemoryUsageSource : uint8_t + { + None, + Cgroups, + Jemalloc + }; + + MemoryUsageSource getSource(); + + void start(); + + ~MemoryWorker(); +private: + uint64_t getMemoryUsage(); + + void backgroundThread(); + + ThreadFromGlobalPool background_thread; + + std::mutex mutex; + std::condition_variable cv; + bool shutdown = false; + + LoggerPtr log; + + uint64_t period_ms; + + MemoryUsageSource source{MemoryUsageSource::None}; + + std::shared_ptr cgroups_reader; + +#if USE_JEMALLOC + JemallocMibCache epoch_mib{"epoch"}; + JemallocMibCache resident_mib{"stats.resident"}; + JemallocMibCache allocated_mib{"stats.allocated"}; + +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + JemallocMibCache purge_mib{"arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"}; +#undef STRINGIFY +#undef STRINGIFY_HELPER +#endif +}; + +} diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index af1b7fbeb4a..467dfe60cd7 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -827,6 +827,9 @@ The server successfully detected this situation and will download merged part fr M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \ M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \ M(GWPAsanFree, "Number of free operations done by GWPAsan") \ + \ + M(MemoryWorkerRun, "Number of runs done by MemoryWorker in background") \ + M(MemoryWorkerRunElapsedMicroseconds, "Total time spent by MemoryWorker for background work") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7cca262baca..34c9d94fca5 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -171,6 +171,7 @@ bool isUserError(Error zk_return_code) void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } +void RemoveRecursiveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ce7489a33e5..a0d6ae54f56 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -248,6 +248,23 @@ struct RemoveResponse : virtual Response { }; +struct RemoveRecursiveRequest : virtual Request +{ + String path; + + /// strict limit for number of deleted nodes + uint32_t remove_nodes_limit = 1; + + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + + size_t bytesSize() const override { return path.size() + sizeof(remove_nodes_limit); } +}; + +struct RemoveRecursiveResponse : virtual Response +{ +}; + struct ExistsRequest : virtual Request { String path; @@ -430,6 +447,7 @@ struct ErrorResponse : virtual Response using CreateCallback = std::function; using RemoveCallback = std::function; +using RemoveRecursiveCallback = std::function; using ExistsCallback = std::function; using GetCallback = std::function; using SetCallback = std::function; @@ -587,6 +605,11 @@ public: int32_t version, RemoveCallback callback) = 0; + virtual void removeRecursive( + const String & path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) = 0; + virtual void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 16ea412eb77..2fbe9110b6b 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -90,6 +90,36 @@ struct TestKeeperRemoveRequest final : RemoveRequest, TestKeeperRequest } }; +struct TestKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, TestKeeperRequest +{ + TestKeeperRemoveRecursiveRequest() = default; + explicit TestKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {} + ResponsePtr createResponse() const override; + std::pair process(TestKeeper::Container & container, int64_t zxid) const override; + + void processWatches(TestKeeper::Watches & node_watches, TestKeeper::Watches & list_watches) const override + { + std::vector> deleted; + + auto add_deleted_watches = [&](TestKeeper::Watches & w) + { + for (const auto & [watch_path, _] : w) + if (watch_path.starts_with(path)) + deleted.emplace_back(watch_path, std::count(watch_path.begin(), watch_path.end(), '/')); + }; + + add_deleted_watches(node_watches); + add_deleted_watches(list_watches); + std::sort(deleted.begin(), deleted.end(), [](const auto & lhs, const auto & rhs) + { + return lhs.second < rhs.second; + }); + + for (const auto & [watch_path, _] : deleted) + processWatchesImpl(watch_path, node_watches, list_watches); + } +}; + struct TestKeeperExistsRequest final : ExistsRequest, TestKeeperRequest { ResponsePtr createResponse() const override; @@ -175,6 +205,10 @@ struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest { requests.push_back(std::make_shared(*concrete_request_remove)); } + else if (const auto * concrete_request_remove_recursive = dynamic_cast(generic_request.get())) + { + requests.push_back(std::make_shared(*concrete_request_remove_recursive)); + } else if (const auto * concrete_request_set = dynamic_cast(generic_request.get())) { requests.push_back(std::make_shared(*concrete_request_set)); @@ -313,6 +347,62 @@ std::pair TestKeeperRemoveRequest::process(TestKeeper::Contai return { std::make_shared(response), undo }; } +std::pair TestKeeperRemoveRecursiveRequest::process(TestKeeper::Container & container, int64_t zxid) const +{ + RemoveRecursiveResponse response; + response.zxid = zxid; + Undo undo; + + auto root_it = container.find(path); + if (root_it == container.end()) + { + response.error = Error::ZNONODE; + return { std::make_shared(response), undo }; + } + + std::vector> children; + + for (auto it = std::next(root_it); it != container.end(); ++it) + { + const auto & [child_path, child_node] = *it; + + if (child_path.starts_with(path)) + children.emplace_back(child_path, child_node); + else + break; + } + + if (children.size() > remove_nodes_limit) + { + response.error = Error::ZNOTEMPTY; + return { std::make_shared(response), undo }; + } + + auto & parent = container.at(parentPath(path)); + --parent.stat.numChildren; + ++parent.stat.cversion; + + for (const auto & [child_path, child_node] : children) + { + auto child_it = container.find(child_path); + chassert(child_it != container.end()); + container.erase(child_it); + } + + response.error = Error::ZOK; + undo = [&container, dead = std::move(children), root_path = path]() + { + for (auto && [child_path, child_node] : dead) + container.emplace(child_path, child_node); + + auto & undo_parent = container.at(parentPath(root_path)); + ++undo_parent.stat.numChildren; + --undo_parent.stat.cversion; + }; + + return { std::make_shared(response), undo }; +} + std::pair TestKeeperExistsRequest::process(TestKeeper::Container & container, int64_t zxid) const { ExistsResponse response; @@ -530,6 +620,7 @@ std::pair TestKeeperMultiRequest::process(TestKeeper::Contain ResponsePtr TestKeeperCreateRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperRemoveRequest::createResponse() const { return std::make_shared(); } +ResponsePtr TestKeeperRemoveRecursiveRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperExistsRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperGetRequest::createResponse() const { return std::make_shared(); } ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shared(); } @@ -771,6 +862,21 @@ void TestKeeper::remove( pushRequest(std::move(request_info)); } +void TestKeeper::removeRecursive( + const String & path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) +{ + TestKeeperRemoveRecursiveRequest request; + request.path = path; + request.remove_nodes_limit = remove_nodes_limit; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + pushRequest(std::move(request_info)); +} + void TestKeeper::exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 562c313ac0e..c32f0064dec 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -58,6 +58,11 @@ public: int32_t version, RemoveCallback callback) override; + void removeRecursive( + const String & path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) override; + void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/ZooKeeper/Types.h b/src/Common/ZooKeeper/Types.h index d2876adaabc..4a163c15838 100644 --- a/src/Common/ZooKeeper/Types.h +++ b/src/Common/ZooKeeper/Types.h @@ -31,6 +31,7 @@ using AsyncResponses = std::vector>>; Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode, bool ignore_if_exists = false); Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version); +Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit); Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version); Coordination::RequestPtr makeCheckRequest(const std::string & path, int version); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 1a9ed4f1ee7..ae60520affb 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -979,18 +979,47 @@ bool ZooKeeper::tryRemoveChildrenRecursive(const std::string & path, bool probab return removed_as_expected; } -void ZooKeeper::removeRecursive(const std::string & path) +void ZooKeeper::removeRecursive(const std::string & path, uint32_t remove_nodes_limit) { - removeChildrenRecursive(path); - remove(path); + if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE)) + { + removeChildrenRecursive(path); + remove(path); + return; + } + + check(tryRemoveRecursive(path, remove_nodes_limit), path); } -void ZooKeeper::tryRemoveRecursive(const std::string & path) +Coordination::Error ZooKeeper::tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit) { - tryRemoveChildrenRecursive(path); - tryRemove(path); -} + if (!isFeatureEnabled(DB::KeeperFeatureFlag::REMOVE_RECURSIVE)) + { + tryRemoveChildrenRecursive(path); + return tryRemove(path); + } + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + auto callback = [promise](const Coordination::RemoveRecursiveResponse & response) mutable + { + promise->set_value(response); + }; + + impl->removeRecursive(path, remove_nodes_limit, std::move(callback)); + + if (future.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready) + { + impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::RemoveRecursive, path)); + return Coordination::Error::ZOPERATIONTIMEOUT; + } + else + { + auto response = future.get(); + return response.error; + } +} namespace { @@ -1619,6 +1648,14 @@ Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version return request; } +Coordination::RequestPtr makeRemoveRecursiveRequest(const std::string & path, uint32_t remove_nodes_limit) +{ + auto request = std::make_shared(); + request->path = path; + request->remove_nodes_limit = remove_nodes_limit; + return request; +} + Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version) { auto request = std::make_shared(); diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 7ccdc9d1b7f..29c4fbc9b74 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -479,15 +479,16 @@ public: Int64 getClientID(); - /// Remove the node with the subtree. If someone concurrently adds or removes a node - /// in the subtree, the result is undefined. - void removeRecursive(const std::string & path); + /// Remove the node with the subtree. + /// If Keeper supports RemoveRecursive operation then it will be performed atomically. + /// Otherwise if someone concurrently adds or removes a node in the subtree, the result is undefined. + void removeRecursive(const std::string & path, uint32_t remove_nodes_limit = 100); - /// Remove the node with the subtree. If someone concurrently removes a node in the subtree, - /// this will not cause errors. + /// Same as removeRecursive but in case if Keeper does not supports RemoveRecursive and + /// if someone concurrently removes a node in the subtree, this will not cause errors. /// For instance, you can call this method twice concurrently for the same node and the end /// result would be the same as for the single call. - void tryRemoveRecursive(const std::string & path); + Coordination::Error tryRemoveRecursive(const std::string & path, uint32_t remove_nodes_limit = 100); /// Similar to removeRecursive(...) and tryRemoveRecursive(...), but does not remove path itself. /// Node defined as RemoveException will not be deleted. diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index dff14f74681..3f9225e84dd 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -1,5 +1,5 @@ -#include "Common/ZooKeeper/IKeeper.h" -#include "Common/ZooKeeper/ZooKeeperConstants.h" +#include +#include #include #include #include @@ -232,6 +232,27 @@ void ZooKeeperRemoveRequest::readImpl(ReadBuffer & in) Coordination::read(version, in); } +void ZooKeeperRemoveRecursiveRequest::writeImpl(WriteBuffer & out) const +{ + Coordination::write(path, out); + Coordination::write(remove_nodes_limit, out); +} + +void ZooKeeperRemoveRecursiveRequest::readImpl(ReadBuffer & in) +{ + Coordination::read(path, in); + Coordination::read(remove_nodes_limit, in); +} + +std::string ZooKeeperRemoveRecursiveRequest::toStringImpl(bool /*short_format*/) const +{ + return fmt::format( + "path = {}\n" + "remove_nodes_limit = {}", + path, + remove_nodes_limit); +} + void ZooKeeperExistsRequest::writeImpl(WriteBuffer & out) const { Coordination::write(path, out); @@ -510,6 +531,11 @@ ZooKeeperMultiRequest::ZooKeeperMultiRequest(std::span(*concrete_request_remove)); } + else if (const auto * concrete_request_remove_recursive = dynamic_cast(generic_request.get())) + { + checkOperationType(Write); + requests.push_back(std::make_shared(*concrete_request_remove_recursive)); + } else if (const auto * concrete_request_set = dynamic_cast(generic_request.get())) { checkOperationType(Write); @@ -707,6 +733,7 @@ ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return se ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperRemoveRecursiveRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } @@ -1024,6 +1051,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory() registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); registerZooKeeperRequest(*this); + registerZooKeeperRequest(*this); } PathMatchResult matchPath(std::string_view path, std::string_view match_to) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index fd6ec3cd375..66c075b277b 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -285,6 +285,31 @@ struct ZooKeeperRemoveResponse final : RemoveResponse, ZooKeeperResponse size_t bytesSize() const override { return RemoveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } }; +struct ZooKeeperRemoveRecursiveRequest final : RemoveRecursiveRequest, ZooKeeperRequest +{ + ZooKeeperRemoveRecursiveRequest() = default; + explicit ZooKeeperRemoveRecursiveRequest(const RemoveRecursiveRequest & base) : RemoveRecursiveRequest(base) {} + + OpNum getOpNum() const override { return OpNum::RemoveRecursive; } + void writeImpl(WriteBuffer & out) const override; + void readImpl(ReadBuffer & in) override; + std::string toStringImpl(bool short_format) const override; + + ZooKeeperResponsePtr makeResponse() const override; + bool isReadRequest() const override { return false; } + + size_t bytesSize() const override { return RemoveRecursiveRequest::bytesSize() + sizeof(xid); } +}; + +struct ZooKeeperRemoveRecursiveResponse : RemoveRecursiveResponse, ZooKeeperResponse +{ + void readImpl(ReadBuffer &) override {} + void writeImpl(WriteBuffer &) const override {} + OpNum getOpNum() const override { return OpNum::RemoveRecursive; } + + size_t bytesSize() const override { return RemoveRecursiveResponse::bytesSize() + sizeof(xid) + sizeof(zxid); } +}; + struct ZooKeeperExistsRequest final : ExistsRequest, ZooKeeperRequest { ZooKeeperExistsRequest() = default; diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.cpp b/src/Common/ZooKeeper/ZooKeeperConstants.cpp index cf8ba35e992..a2780dfd5e2 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.cpp +++ b/src/Common/ZooKeeper/ZooKeeperConstants.cpp @@ -29,6 +29,7 @@ static const std::unordered_set VALID_OPERATIONS = static_cast(OpNum::GetACL), static_cast(OpNum::FilteredList), static_cast(OpNum::CheckNotExists), + static_cast(OpNum::RemoveRecursive), }; OpNum getOpNum(int32_t raw_op_num) diff --git a/src/Common/ZooKeeper/ZooKeeperConstants.h b/src/Common/ZooKeeper/ZooKeeperConstants.h index 1d9830505f8..9d8e2d4f857 100644 --- a/src/Common/ZooKeeper/ZooKeeperConstants.h +++ b/src/Common/ZooKeeper/ZooKeeperConstants.h @@ -40,6 +40,7 @@ enum class OpNum : int32_t FilteredList = 500, CheckNotExists = 501, CreateIfNotExists = 502, + RemoveRecursive = 503, SessionID = 997, /// Special internal request }; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index ba622f30c91..a6dd9738e17 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1347,6 +1347,25 @@ void ZooKeeper::remove( ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); } +void ZooKeeper::removeRecursive( + const String &path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) +{ + if (!isFeatureEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE)) + throw Exception::fromMessage(Error::ZBADARGUMENTS, "RemoveRecursive request type cannot be used because it's not supported by the server"); + + ZooKeeperRemoveRecursiveRequest request; + request.path = path; + request.remove_nodes_limit = remove_nodes_limit; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; + + pushRequest(std::move(request_info)); + ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); +} void ZooKeeper::exists( const String & path, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 39082cd14c1..47d2ab8f401 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -146,6 +146,11 @@ public: int32_t version, RemoveCallback callback) override; + void removeRecursive( + const String &path, + uint32_t remove_nodes_limit, + RemoveRecursiveCallback callback) override; + void exists( const String & path, ExistsCallback callback, diff --git a/src/Common/tests/gtest_cgroups_reader.cpp b/src/Common/tests/gtest_cgroups_reader.cpp index 2de25bb42ce..e24b91a59b8 100644 --- a/src/Common/tests/gtest_cgroups_reader.cpp +++ b/src/Common/tests/gtest_cgroups_reader.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include using namespace DB; @@ -126,7 +126,7 @@ const std::string EXPECTED[2] "\"workingset_restore_anon\": 0, \"workingset_restore_file\": 0, \"zswap\": 0, \"zswapped\": 0, \"zswpin\": 0, \"zswpout\": 0}"}; -class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam +class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParam { void SetUp() override { @@ -138,7 +138,7 @@ class CgroupsMemoryUsageObserverFixture : public ::testing::TestWithParamreadMemoryUsage(), - version == CgroupsMemoryUsageObserver::CgroupsVersion::V1 ? /* rss from memory.stat */ 2232029184 - : /* value from memory.current - inactive_file */ 20952338432); + version == ICgroupsReader::CgroupsVersion::V1 ? /* rss from memory.stat */ 2232029184 + : /* anon from memory.stat */ 10429399040); } TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest) { const auto version = GetParam(); - auto reader = createCgroupsReader(version, tmp_dir); + auto reader = ICgroupsReader::createCgroupsReader(version, tmp_dir); ASSERT_EQ(reader->dumpAllStats(), EXPECTED[static_cast(version)]); } @@ -173,6 +173,6 @@ TEST_P(CgroupsMemoryUsageObserverFixture, DumpAllStatsTest) INSTANTIATE_TEST_SUITE_P( CgroupsMemoryUsageObserverTests, CgroupsMemoryUsageObserverFixture, - ::testing::Values(CgroupsMemoryUsageObserver::CgroupsVersion::V1, CgroupsMemoryUsageObserver::CgroupsVersion::V2)); + ::testing::Values(ICgroupsReader::CgroupsVersion::V1, ICgroupsReader::CgroupsVersion::V2)); #endif diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index e416fadc829..907e87a6d30 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -39,7 +39,7 @@ using Checksum = CityHash_v1_0_2::uint128; /// Validate checksum of data, and if it mismatches, find out possible reason and throw exception. -static void validateChecksum(char * data, size_t size, const Checksum expected_checksum) +static void validateChecksum(char * data, size_t size, const Checksum expected_checksum, bool external_data) { auto calculated_checksum = CityHash_v1_0_2::CityHash128(data, size); if (expected_checksum == calculated_checksum) @@ -64,6 +64,8 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c "this can be caused by disk bit rot. This exception protects ClickHouse " "from data corruption due to hardware failures."; + int error_code = external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CHECKSUM_DOESNT_MATCH; + auto flip_bit = [](char * buf, size_t pos) { buf[pos / 8] ^= 1 << pos % 8; @@ -87,7 +89,7 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c { message << ". The mismatch is caused by single bit flip in data block at byte " << (bit_pos / 8) << ", bit " << (bit_pos % 8) << ". " << message_hardware_failure; - throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); + throw Exception::createDeprecated(message.str(), error_code); } flip_bit(tmp_data, bit_pos); /// Restore @@ -102,10 +104,10 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c { message << ". The mismatch is caused by single bit flip in checksum. " << message_hardware_failure; - throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); + throw Exception::createDeprecated(message.str(), error_code); } - throw Exception::createDeprecated(message.str(), ErrorCodes::CHECKSUM_DOESNT_MATCH); + throw Exception::createDeprecated(message.str(), error_code); } static void readHeaderAndGetCodecAndSize( @@ -151,7 +153,7 @@ static void readHeaderAndGetCodecAndSize( "Most likely corrupted data.", size_compressed_without_checksum); if (size_compressed_without_checksum < header_size) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Can't decompress data: " + throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA, "Can't decompress data: " "the compressed data size ({}, this should include header size) is less than the header size ({})", size_compressed_without_checksum, static_cast(header_size)); } @@ -202,7 +204,7 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, readBinaryLittleEndian(checksum.low64, checksum_in); readBinaryLittleEndian(checksum.high64, checksum_in); - validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); + validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data); } ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); @@ -247,7 +249,7 @@ size_t CompressedReadBufferBase::readCompressedDataBlockForAsynchronous(size_t & readBinaryLittleEndian(checksum.low64, checksum_in); readBinaryLittleEndian(checksum.high64, checksum_in); - validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); + validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum, external_data); } ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed_without_checksum + sizeof(Checksum)); @@ -307,7 +309,7 @@ void CompressedReadBufferBase::decompress(BufferBase::Buffer & to, size_t size_d UInt8 header_size = ICompressionCodec::getHeaderSize(); if (size_compressed_without_checksum < header_size) - throw Exception(ErrorCodes::CORRUPTED_DATA, + throw Exception(external_data ? ErrorCodes::CANNOT_DECOMPRESS : ErrorCodes::CORRUPTED_DATA, "Can't decompress data: the compressed data size ({}, this should include header size) is less than the header size ({})", size_compressed_without_checksum, static_cast(header_size)); diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..157858f3c44 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -114,8 +114,13 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) + , context(std::move(context_)) { } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index ec0e60cbb6e..a2ab7cab756 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -13,9 +13,13 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); - ~KeeperAsynchronousMetrics() override; + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~KeeperAsynchronousMetrics() override; private: ContextPtr context; diff --git a/src/Coordination/KeeperConstants.h b/src/Coordination/KeeperConstants.h index 08a7c85585a..d984d077872 100644 --- a/src/Coordination/KeeperConstants.h +++ b/src/Coordination/KeeperConstants.h @@ -11,6 +11,7 @@ enum class KeeperApiVersion : uint8_t WITH_FILTERED_LIST, WITH_MULTI_READ, WITH_CHECK_NOT_EXISTS, + WITH_REMOVE_RECURSIVE, }; const String keeper_system_path = "/keeper"; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 4a350077596..142bd3b7c71 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -91,6 +91,12 @@ bool checkIfRequestIncreaseMem(const Coordination::ZooKeeperRequestPtr & request memory_delta -= remove_req.bytesSize(); break; } + case Coordination::OpNum::RemoveRecursive: + { + Coordination::ZooKeeperRemoveRecursiveRequest & remove_req = dynamic_cast(*sub_zk_request); + memory_delta -= remove_req.bytesSize(); + break; + } default: break; } @@ -148,7 +154,14 @@ void KeeperDispatcher::requestThread() Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); if (configuration_and_settings->standalone_keeper && isExceedingMemorySoftLimit() && checkIfRequestIncreaseMem(request.request)) { - LOG_WARNING(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), request.request->getOpNum()); + LOG_WARNING( + log, + "Processing requests refused because of max_memory_usage_soft_limit {}, the total allocated memory is {}, RSS is {}, request type " + "is {}", + ReadableSize(mem_soft_limit), + ReadableSize(total_memory_tracker.get()), + ReadableSize(total_memory_tracker.getRSS()), + request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } diff --git a/src/Coordination/KeeperFeatureFlags.h b/src/Coordination/KeeperFeatureFlags.h index 4e26ca60736..e70bd50cc88 100644 --- a/src/Coordination/KeeperFeatureFlags.h +++ b/src/Coordination/KeeperFeatureFlags.h @@ -12,6 +12,7 @@ enum class KeeperFeatureFlag : size_t MULTI_READ, CHECK_NOT_EXISTS, CREATE_IF_NOT_EXISTS, + REMOVE_RECURSIVE, }; class KeeperFeatureFlags diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index f09ea56391a..918f24efb2c 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -602,7 +602,7 @@ bool KeeperServer::isLeaderAlive() const bool KeeperServer::isExceedingMemorySoftLimit() const { Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); - return mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit; + return mem_soft_limit > 0 && std::max(total_memory_tracker.get(), total_memory_tracker.getRSS()) >= mem_soft_limit; } /// TODO test whether taking failed peer in count diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index acdf209baae..2eb9ab30efa 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -832,6 +832,15 @@ std::shared_ptr KeeperStorage::UncommittedS return tryGetNodeFromStorage(path); } +template +const typename Container::Node * KeeperStorage::UncommittedState::getActualNodeView(StringRef path, const Node & storage_node) const +{ + if (auto node_it = nodes.find(path.toView()); node_it != nodes.end()) + return node_it->second.node.get(); + + return &storage_node; +} + template Coordination::ACLs KeeperStorage::UncommittedState::getACLs(StringRef path) const { @@ -1124,7 +1133,7 @@ struct KeeperStorageRequestProcessor } virtual KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const + processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & /*watches*/, KeeperStorageBase::Watches & /*list_watches*/) const { return {}; } @@ -1241,7 +1250,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override + processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override { return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED); } @@ -1462,16 +1471,41 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce } }; +namespace +{ + +template +void addUpdateParentPzxidDelta(Storage & storage, std::vector & deltas, int64_t zxid, StringRef path) +{ + auto parent_path = parentNodePath(path); + if (!storage.uncommitted_state.getNode(parent_path)) + return; + + deltas.emplace_back( + std::string{parent_path}, + zxid, + typename Storage::UpdateNodeDelta + { + [zxid](Storage::Node & parent) + { + parent.pzxid = std::max(parent.pzxid, zxid); + } + } + ); +} + +} + template struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestProcessor { + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override { return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); } - using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; - std::vector preprocess(Storage & storage, int64_t zxid, int64_t /*session_id*/, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override { @@ -1488,31 +1522,12 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; } - const auto update_parent_pzxid = [&]() - { - auto parent_path = parentNodePath(request.path); - if (!storage.uncommitted_state.getNode(parent_path)) - return; - - new_deltas.emplace_back( - std::string{parent_path}, - zxid, - typename Storage::UpdateNodeDelta - { - [zxid](Storage::Node & parent) - { - parent.pzxid = std::max(parent.pzxid, zxid); - } - } - ); - }; - auto node = storage.uncommitted_state.getNode(request.path); if (!node) { if (request.restored_from_zookeeper_log) - update_parent_pzxid(); + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; } else if (request.version != -1 && request.version != node->version) @@ -1521,7 +1536,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; if (request.restored_from_zookeeper_log) - update_parent_pzxid(); + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); new_deltas.emplace_back( std::string{parentNodePath(request.path)}, @@ -1552,12 +1567,318 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr } KeeperStorageBase::ResponsesForSessions - processWatches(KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override + processWatches(const Storage & /*storage*/, int64_t /*zxid*/, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override { return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED); } }; +template +struct KeeperStorageRemoveRecursiveRequestProcessor final : public KeeperStorageRequestProcessor +{ + using KeeperStorageRequestProcessor::KeeperStorageRequestProcessor; + + bool checkAuth(Storage & storage, int64_t session_id, bool is_local) const override + { + return storage.checkACL(parentNodePath(this->zk_request->getPath()), Coordination::ACL::Delete, session_id, is_local); + } + + std::vector + preprocess(Storage & storage, int64_t zxid, int64_t session_id, int64_t /*time*/, uint64_t & digest, const KeeperContext & keeper_context) const override + { + ProfileEvents::increment(ProfileEvents::KeeperRemoveRequest); + Coordination::ZooKeeperRemoveRecursiveRequest & request = dynamic_cast(*this->zk_request); + + std::vector new_deltas; + + if (Coordination::matchPath(request.path, keeper_system_path) != Coordination::PathMatchResult::NOT_MATCH) + { + auto error_msg = fmt::format("Trying to delete an internal Keeper path ({}) which is not allowed", request.path); + + handleSystemNodeModification(keeper_context, error_msg); + return {typename Storage::Delta{zxid, Coordination::Error::ZBADARGUMENTS}}; + } + + auto node = storage.uncommitted_state.getNode(request.path); + + if (!node) + { + if (request.restored_from_zookeeper_log) + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); + + return {typename Storage::Delta{zxid, Coordination::Error::ZNONODE}}; + } + + ToDeleteTreeCollector collector(storage, zxid, session_id, request.remove_nodes_limit); + auto collect_status = collector.collect(request.path, *node); + + if (collect_status == ToDeleteTreeCollector::CollectStatus::NoAuth) + return {typename Storage::Delta{zxid, Coordination::Error::ZNOAUTH}}; + + if (collect_status == ToDeleteTreeCollector::CollectStatus::LimitExceeded) + return {typename Storage::Delta{zxid, Coordination::Error::ZNOTEMPTY}}; + + if (request.restored_from_zookeeper_log) + addUpdateParentPzxidDelta(storage, new_deltas, zxid, request.path); + + auto delete_deltas = collector.extractDeltas(); + + for (const auto & delta : delete_deltas) + { + const auto * remove_delta = std::get_if(&delta.operation); + if (remove_delta && remove_delta->ephemeral_owner) + storage.unregisterEphemeralPath(remove_delta->ephemeral_owner, delta.path); + } + + new_deltas.insert(new_deltas.end(), std::make_move_iterator(delete_deltas.begin()), std::make_move_iterator(delete_deltas.end())); + + digest = storage.calculateNodesDigest(digest, new_deltas); + + return new_deltas; + } + + Coordination::ZooKeeperResponsePtr process(Storage & storage, int64_t zxid) const override + { + Coordination::ZooKeeperResponsePtr response_ptr = this->zk_request->makeResponse(); + Coordination::ZooKeeperRemoveRecursiveResponse & response = dynamic_cast(*response_ptr); + + response.error = storage.commit(zxid); + return response_ptr; + } + + KeeperStorageBase::ResponsesForSessions + processWatches(const Storage & storage, int64_t zxid, KeeperStorageBase::Watches & watches, KeeperStorageBase::Watches & list_watches) const override + { + /// need to iterate over zxid deltas and update watches for deleted tree. + const auto & deltas = storage.uncommitted_state.deltas; + + KeeperStorageBase::ResponsesForSessions responses; + for (auto it = deltas.rbegin(); it != deltas.rend() && it->zxid == zxid; ++it) + { + const auto * remove_delta = std::get_if(&it->operation); + if (remove_delta) + { + auto new_responses = processWatchesImpl(it->path, watches, list_watches, Coordination::Event::DELETED); + responses.insert(responses.end(), std::make_move_iterator(new_responses.begin()), std::make_move_iterator(new_responses.end())); + } + } + + return responses; + } + +private: + using SNode = typename Storage::Node; + + class ToDeleteTreeCollector + { + Storage & storage; + int64_t zxid; + int64_t session_id; + uint32_t limit; + + uint32_t max_level = 0; + uint32_t nodes_observed = 1; /// root node + std::unordered_map> by_level_deltas; + + struct Step + { + String path; + std::variant node; + uint32_t level; + }; + + enum class CollectStatus + { + Ok, + NoAuth, + LimitExceeded, + }; + + friend struct KeeperStorageRemoveRecursiveRequestProcessor; + + public: + ToDeleteTreeCollector(Storage & storage_, int64_t zxid_, int64_t session_id_, uint32_t limit_) + : storage(storage_) + , zxid(zxid_) + , session_id(session_id_) + , limit(limit_) + { + } + + CollectStatus collect(StringRef root_path, const SNode & root_node) + { + std::deque steps; + + if (checkLimits(&root_node)) + return CollectStatus::LimitExceeded; + + steps.push_back(Step{root_path.toString(), &root_node, 0}); + + while (!steps.empty()) + { + Step step = std::move(steps.front()); + steps.pop_front(); + + StringRef path = step.path; + uint32_t level = step.level; + const SNode * node_ptr = nullptr; + + if (auto * rdb = std::get_if(&step.node)) + node_ptr = rdb; + else + node_ptr = std::get(step.node); + + chassert(!path.empty()); + chassert(node_ptr != nullptr); + + const auto & node = *node_ptr; + auto actual_node_ptr = storage.uncommitted_state.getActualNodeView(path, node); + chassert(actual_node_ptr != nullptr); /// explicitly check that node is not deleted + + if (actual_node_ptr->numChildren() > 0 && !storage.checkACL(path, Coordination::ACL::Delete, session_id, /*is_local=*/false)) + return CollectStatus::NoAuth; + + if (auto status = visitRocksDBNode(steps, path, level); status != CollectStatus::Ok) + return status; + + if (auto status = visitMemNode(steps, path, level); status != CollectStatus::Ok) + return status; + + if (auto status = visitRootAndUncommitted(steps, path, node, level); status != CollectStatus::Ok) + return status; + } + + return CollectStatus::Ok; + } + + std::vector extractDeltas() + { + std::vector deltas; + + for (ssize_t level = max_level; level >= 0; --level) + { + auto & level_deltas = by_level_deltas[static_cast(level)]; + deltas.insert(deltas.end(), std::make_move_iterator(level_deltas.begin()), std::make_move_iterator(level_deltas.end())); + } + + return std::move(deltas); + } + + private: + CollectStatus visitRocksDBNode(std::deque & steps, StringRef root_path, uint32_t level) + { + if constexpr (Storage::use_rocksdb) + { + std::filesystem::path root_fs_path(root_path.toString()); + auto children = storage.container.getChildren(root_path.toString()); + + for (auto && [child_name, child_node] : children) + { + auto child_path = (root_fs_path / child_name).generic_string(); + const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); + + if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + continue; + + if (checkLimits(actual_child_node_ptr)) + return CollectStatus::LimitExceeded; + + steps.push_back(Step{std::move(child_path), std::move(child_node), level + 1}); + } + } + + return CollectStatus::Ok; + } + + CollectStatus visitMemNode(std::deque & steps, StringRef root_path, uint32_t level) + { + if constexpr (!Storage::use_rocksdb) + { + auto node_it = storage.container.find(root_path); + if (node_it == storage.container.end()) + return CollectStatus::Ok; + + std::filesystem::path root_fs_path(root_path.toString()); + const auto & children = node_it->value.getChildren(); + + for (const auto & child_name : children) + { + auto child_path = (root_fs_path / child_name.toView()).generic_string(); + + auto child_it = storage.container.find(child_path); + chassert(child_it != storage.container.end()); + const auto & child_node = child_it->value; + + const auto actual_child_node_ptr = storage.uncommitted_state.getActualNodeView(child_path, child_node); + + if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + continue; + + if (checkLimits(actual_child_node_ptr)) + return CollectStatus::LimitExceeded; + + steps.push_back(Step{std::move(child_path), &child_node, level + 1}); + } + } + + return CollectStatus::Ok; + } + + CollectStatus visitRootAndUncommitted(std::deque & steps, StringRef root_path, const SNode & root_node, uint32_t level) + { + const auto & nodes = storage.uncommitted_state.nodes; + + /// nodes are sorted by paths with level locality + auto it = nodes.upper_bound(root_path.toString() + "/"); + + for (; it != nodes.end() && parentNodePath(it->first) == root_path; ++it) + { + const auto actual_child_node_ptr = it->second.node.get(); + + if (actual_child_node_ptr == nullptr) /// node was deleted in previous step of multi transaction + continue; + + if (checkLimits(actual_child_node_ptr)) + return CollectStatus::LimitExceeded; + + const String & child_path = it->first; + const SNode & child_node = *it->second.node; + + steps.push_back(Step{child_path, &child_node, level + 1}); + } + + addDelta(root_path, root_node, level); + + return CollectStatus::Ok; + } + + void addDelta(StringRef root_path, const SNode & root_node, uint32_t level) + { + max_level = std::max(max_level, level); + + by_level_deltas[level].emplace_back( + parentNodePath(root_path).toString(), + zxid, + typename Storage::UpdateNodeDelta{ + [](SNode & parent) + { + ++parent.cversion; + parent.decreaseNumChildren(); + } + }); + + by_level_deltas[level].emplace_back(root_path.toString(), zxid, typename Storage::RemoveNodeDelta{root_node.version, root_node.ephemeralOwner()}); + } + + bool checkLimits(const SNode * node) + { + chassert(node != nullptr); + nodes_observed += node->numChildren(); + return nodes_observed > limit; + } + }; +}; + template struct KeeperStorageExistsRequestProcessor final : public KeeperStorageRequestProcessor { @@ -1709,7 +2030,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce } KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override + processWatches(const Storage & /*storage*/, int64_t /*zxid*/, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { return processWatchesImpl(this->zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED); } @@ -2131,6 +2452,10 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro check_operation_type(OperationType::Write); concrete_requests.push_back(std::make_shared>(sub_zk_request)); break; + case Coordination::OpNum::RemoveRecursive: + check_operation_type(OperationType::Write); + concrete_requests.push_back(std::make_shared>(sub_zk_request)); + break; case Coordination::OpNum::Set: check_operation_type(OperationType::Write); concrete_requests.push_back(std::make_shared>(sub_zk_request)); @@ -2250,12 +2575,12 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro } KeeperStorageBase::ResponsesForSessions - processWatches(typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override + processWatches(const Storage & storage, int64_t zxid, typename Storage::Watches & watches, typename Storage::Watches & list_watches) const override { typename Storage::ResponsesForSessions result; for (const auto & generic_request : concrete_requests) { - auto responses = generic_request->processWatches(watches, list_watches); + auto responses = generic_request->processWatches(storage, zxid, watches, list_watches); result.insert(result.end(), responses.begin(), responses.end()); } return result; @@ -2400,6 +2725,7 @@ KeeperStorageRequestProcessorsFactory::KeeperStorageRequestProcessorsFa registerKeeperRequestProcessor>(*this); registerKeeperRequestProcessor>(*this); registerKeeperRequestProcessor>(*this); + registerKeeperRequestProcessor>(*this); } @@ -2718,7 +3044,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::process /// If this requests processed successfully we need to check watches if (response->error == Coordination::Error::ZOK) { - auto watch_responses = request_processor->processWatches(watches, list_watches); + auto watch_responses = request_processor->processWatches(*this, zxid, watches, list_watches); results.insert(results.end(), watch_responses.begin(), watch_responses.end()); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 4a9286d4835..904af76ef37 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -566,6 +566,7 @@ public: void rollback(int64_t rollback_zxid); std::shared_ptr getNode(StringRef path) const; + const Node * getActualNodeView(StringRef path, const Node & storage_node) const; Coordination::ACLs getACLs(StringRef path) const; void applyDelta(const Delta & delta); @@ -609,7 +610,18 @@ public: using is_transparent = void; // required to make find() work with different type than key_type }; - mutable std::unordered_map nodes; + struct PathCmp + { + using is_transparent = std::true_type; + + auto operator()(const std::string_view a, + const std::string_view b) const + { + return a.size() < b.size() || (a.size() == b.size() && a < b); + } + }; + + mutable std::map nodes; std::unordered_map, Hash, Equal> deltas_for_path; std::list deltas; diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d39031773cd..73402af5ec4 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -3113,6 +3113,8 @@ TYPED_TEST(CoordinationTest, TestFeatureFlags) ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::FILTERED_LIST)); ASSERT_TRUE(feature_flags.isEnabled(KeeperFeatureFlag::MULTI_READ)); ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CHECK_NOT_EXISTS)); + ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::CREATE_IF_NOT_EXISTS)); + ASSERT_FALSE(feature_flags.isEnabled(KeeperFeatureFlag::REMOVE_RECURSIVE)); } TYPED_TEST(CoordinationTest, TestSystemNodeModify) @@ -3374,6 +3376,474 @@ TYPED_TEST(CoordinationTest, TestReapplyingDeltas) ASSERT_TRUE(children1_set == children2_set); } +TYPED_TEST(CoordinationTest, TestRemoveRecursiveRequest) +{ + using namespace DB; + using namespace Coordination; + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; + + int32_t zxid = 0; + + const auto create = [&](const String & path, int create_mode) + { + int new_zxid = ++zxid; + + const auto create_request = std::make_shared(); + create_request->path = path; + create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential; + create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential; + + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; + }; + + const auto remove = [&](const String & path, int32_t version = -1) + { + int new_zxid = ++zxid; + + auto remove_request = std::make_shared(); + remove_request->path = path; + remove_request->version = version; + + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + return storage.processRequest(remove_request, 1, new_zxid); + }; + + const auto remove_recursive = [&](const String & path, uint32_t remove_nodes_limit = 1) + { + int new_zxid = ++zxid; + + auto remove_request = std::make_shared(); + remove_request->path = path; + remove_request->remove_nodes_limit = remove_nodes_limit; + + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + return storage.processRequest(remove_request, 1, new_zxid); + }; + + const auto exists = [&](const String & path) + { + int new_zxid = ++zxid; + + const auto exists_request = std::make_shared(); + exists_request->path = path; + + storage.preprocessRequest(exists_request, 1, 0, new_zxid); + auto responses = storage.processRequest(exists_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + return responses[0].response->error == Coordination::Error::ZOK; + }; + + { + SCOPED_TRACE("Single Remove Single Node"); + create("/T1", zkutil::CreateMode::Persistent); + + auto responses = remove("/T1"); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); + ASSERT_FALSE(exists("/T1")); + } + + { + SCOPED_TRACE("Single Remove Tree"); + create("/T2", zkutil::CreateMode::Persistent); + create("/T2/A", zkutil::CreateMode::Persistent); + + auto responses = remove("/T2"); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY); + ASSERT_TRUE(exists("/T2")); + } + + { + SCOPED_TRACE("Recursive Remove Single Node"); + create("/T3", zkutil::CreateMode::Persistent); + + auto responses = remove_recursive("/T3", 100); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); + ASSERT_FALSE(exists("/T3")); + } + + { + SCOPED_TRACE("Recursive Remove Tree Small Limit"); + create("/T5", zkutil::CreateMode::Persistent); + create("/T5/A", zkutil::CreateMode::Persistent); + create("/T5/B", zkutil::CreateMode::Persistent); + create("/T5/A/C", zkutil::CreateMode::Persistent); + + auto responses = remove_recursive("/T5", 2); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZNOTEMPTY); + ASSERT_TRUE(exists("/T5")); + ASSERT_TRUE(exists("/T5/A")); + ASSERT_TRUE(exists("/T5/B")); + ASSERT_TRUE(exists("/T5/A/C")); + } + + { + SCOPED_TRACE("Recursive Remove Tree Big Limit"); + create("/T6", zkutil::CreateMode::Persistent); + create("/T6/A", zkutil::CreateMode::Persistent); + create("/T6/B", zkutil::CreateMode::Persistent); + create("/T6/A/C", zkutil::CreateMode::Persistent); + + auto responses = remove_recursive("/T6", 4); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); + ASSERT_FALSE(exists("/T6")); + ASSERT_FALSE(exists("/T6/A")); + ASSERT_FALSE(exists("/T6/B")); + ASSERT_FALSE(exists("/T6/A/C")); + } + + { + SCOPED_TRACE("Recursive Remove Ephemeral"); + create("/T7", zkutil::CreateMode::Ephemeral); + ASSERT_EQ(storage.ephemerals.size(), 1); + + auto responses = remove_recursive("/T7", 100); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); + ASSERT_EQ(storage.ephemerals.size(), 0); + ASSERT_FALSE(exists("/T7")); + } + + { + SCOPED_TRACE("Recursive Remove Tree With Ephemeral"); + create("/T8", zkutil::CreateMode::Persistent); + create("/T8/A", zkutil::CreateMode::Persistent); + create("/T8/B", zkutil::CreateMode::Ephemeral); + create("/T8/A/C", zkutil::CreateMode::Ephemeral); + ASSERT_EQ(storage.ephemerals.size(), 1); + + auto responses = remove_recursive("/T8", 4); + ASSERT_EQ(responses.size(), 1); + ASSERT_EQ(responses[0].response->error, Coordination::Error::ZOK); + ASSERT_EQ(storage.ephemerals.size(), 0); + ASSERT_FALSE(exists("/T8")); + ASSERT_FALSE(exists("/T8/A")); + ASSERT_FALSE(exists("/T8/B")); + ASSERT_FALSE(exists("/T8/A/C")); + } +} + +TYPED_TEST(CoordinationTest, TestRemoveRecursiveInMultiRequest) +{ + using namespace DB; + using namespace Coordination; + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; + int zxid = 0; + + auto prepare_create_tree = []() + { + return Coordination::Requests{ + zkutil::makeCreateRequest("/A", "A", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/B", "B", zkutil::CreateMode::Persistent), + zkutil::makeCreateRequest("/A/C", "C", zkutil::CreateMode::Ephemeral), + zkutil::makeCreateRequest("/A/B/D", "D", zkutil::CreateMode::Ephemeral), + }; + }; + + const auto exists = [&](const String & path) + { + int new_zxid = ++zxid; + + const auto exists_request = std::make_shared(); + exists_request->path = path; + + storage.preprocessRequest(exists_request, 1, 0, new_zxid); + auto responses = storage.processRequest(exists_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + return responses[0].response->error == Coordination::Error::ZOK; + }; + + const auto is_multi_ok = [&](Coordination::ZooKeeperResponsePtr response) + { + const auto & multi_response = dynamic_cast(*response); + + for (const auto & op_response : multi_response.responses) + if (op_response->error != Coordination::Error::ZOK) + return false; + + return true; + }; + + { + SCOPED_TRACE("Remove In Multi Tx"); + int new_zxid = ++zxid; + auto ops = prepare_create_tree(); + + ops.push_back(zkutil::makeRemoveRequest("/A", -1)); + const auto request = std::make_shared(ops, ACLs{}); + + storage.preprocessRequest(request, 1, 0, new_zxid); + auto responses = storage.processRequest(request, 1, new_zxid); + ops.pop_back(); + + ASSERT_EQ(responses.size(), 1); + ASSERT_FALSE(is_multi_ok(responses[0].response)); + } + + { + SCOPED_TRACE("Recursive Remove In Multi Tx"); + int new_zxid = ++zxid; + auto ops = prepare_create_tree(); + + ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 4)); + const auto request = std::make_shared(ops, ACLs{}); + + storage.preprocessRequest(request, 1, 0, new_zxid); + auto responses = storage.processRequest(request, 1, new_zxid); + ops.pop_back(); + + ASSERT_EQ(responses.size(), 1); + ASSERT_TRUE(is_multi_ok(responses[0].response)); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/C")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/B/D")); + } + + { + SCOPED_TRACE("Recursive Remove With Regular In Multi Tx"); + int new_zxid = ++zxid; + auto ops = prepare_create_tree(); + + ops.push_back(zkutil::makeRemoveRequest("/A/C", -1)); + ops.push_back(zkutil::makeRemoveRecursiveRequest("/A", 3)); + const auto request = std::make_shared(ops, ACLs{}); + + storage.preprocessRequest(request, 1, 0, new_zxid); + auto responses = storage.processRequest(request, 1, new_zxid); + ops.pop_back(); + ops.pop_back(); + + ASSERT_EQ(responses.size(), 1); + ASSERT_TRUE(is_multi_ok(responses[0].response)); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/C")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/B/D")); + } + + { + SCOPED_TRACE("Recursive Remove From Committed and Uncommitted states"); + int create_zxid = ++zxid; + auto ops = prepare_create_tree(); + + /// First create nodes + const auto create_request = std::make_shared(ops, ACLs{}); + storage.preprocessRequest(create_request, 1, 0, create_zxid); + auto create_responses = storage.processRequest(create_request, 1, create_zxid); + ASSERT_EQ(create_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(create_responses[0].response)); + ASSERT_TRUE(exists("/A")); + ASSERT_TRUE(exists("/A/C")); + ASSERT_TRUE(exists("/A/B")); + ASSERT_TRUE(exists("/A/B/D")); + + /// Remove node A/C as a single remove request. + /// Remove all other as remove recursive request. + /// In this case we should list storage to understand the tree topology + /// but ignore already deleted nodes in uncommitted state. + + int remove_zxid = ++zxid; + ops = { + zkutil::makeRemoveRequest("/A/C", -1), + zkutil::makeRemoveRecursiveRequest("/A", 3), + }; + const auto remove_request = std::make_shared(ops, ACLs{}); + + storage.preprocessRequest(remove_request, 1, 0, remove_zxid); + auto remove_responses = storage.processRequest(remove_request, 1, remove_zxid); + + ASSERT_EQ(remove_responses.size(), 1); + ASSERT_TRUE(is_multi_ok(remove_responses[0].response)); + ASSERT_FALSE(exists("/A")); + ASSERT_FALSE(exists("/A/C")); + ASSERT_FALSE(exists("/A/B")); + ASSERT_FALSE(exists("/A/B/D")); + } +} + +TYPED_TEST(CoordinationTest, TestRemoveRecursiveWatches) +{ + using namespace DB; + using namespace Coordination; + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; + int zxid = 0; + + const auto create = [&](const String & path, int create_mode) + { + int new_zxid = ++zxid; + + const auto create_request = std::make_shared(); + create_request->path = path; + create_request->is_ephemeral = create_mode == zkutil::CreateMode::Ephemeral || create_mode == zkutil::CreateMode::EphemeralSequential; + create_request->is_sequential = create_mode == zkutil::CreateMode::PersistentSequential || create_mode == zkutil::CreateMode::EphemeralSequential; + + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; + }; + + const auto add_watch = [&](const String & path) + { + int new_zxid = ++zxid; + + const auto exists_request = std::make_shared(); + exists_request->path = path; + exists_request->has_watch = true; + + storage.preprocessRequest(exists_request, 1, 0, new_zxid); + auto responses = storage.processRequest(exists_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK); + }; + + const auto add_list_watch = [&](const String & path) + { + int new_zxid = ++zxid; + + const auto list_request = std::make_shared(); + list_request->path = path; + list_request->has_watch = true; + + storage.preprocessRequest(list_request, 1, 0, new_zxid); + auto responses = storage.processRequest(list_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK); + }; + + create("/A", zkutil::CreateMode::Persistent); + create("/A/B", zkutil::CreateMode::Persistent); + create("/A/C", zkutil::CreateMode::Ephemeral); + create("/A/B/D", zkutil::CreateMode::Ephemeral); + + add_watch("/A"); + add_watch("/A/B"); + add_watch("/A/C"); + add_watch("/A/B/D"); + add_list_watch("/A"); + add_list_watch("/A/B"); + ASSERT_EQ(storage.watches.size(), 4); + ASSERT_EQ(storage.list_watches.size(), 2); + + int new_zxid = ++zxid; + + auto remove_request = std::make_shared(); + remove_request->path = "/A"; + remove_request->remove_nodes_limit = 4; + + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + auto responses = storage.processRequest(remove_request, 1, new_zxid); + + ASSERT_EQ(responses.size(), 7); + + for (size_t i = 0; i < 7; ++i) + { + ASSERT_EQ(responses[i].response->error, Coordination::Error::ZOK); + + if (const auto * watch_response = dynamic_cast(responses[i].response.get())) + ASSERT_EQ(watch_response->type, Coordination::Event::DELETED); + } + + ASSERT_EQ(storage.watches.size(), 0); + ASSERT_EQ(storage.list_watches.size(), 0); +} + +TYPED_TEST(CoordinationTest, TestRemoveRecursiveAcls) +{ + using namespace DB; + using namespace Coordination; + + using Storage = typename TestFixture::Storage; + + ChangelogDirTest rocks("./rocksdb"); + this->setRocksDBDirectory("./rocksdb"); + + Storage storage{500, "", this->keeper_context}; + int zxid = 0; + + { + int new_zxid = ++zxid; + String user_auth_data = "test_user:test_password"; + + const auto auth_request = std::make_shared(); + auth_request->scheme = "digest"; + auth_request->data = user_auth_data; + + storage.preprocessRequest(auth_request, 1, 0, new_zxid); + auto responses = storage.processRequest(auth_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to add auth to session"; + } + + const auto create = [&](const String & path) + { + int new_zxid = ++zxid; + + const auto create_request = std::make_shared(); + create_request->path = path; + create_request->acls = {{.permissions = ACL::Create, .scheme = "auth", .id = ""}}; + + storage.preprocessRequest(create_request, 1, 0, new_zxid); + auto responses = storage.processRequest(create_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZOK) << "Failed to create " << path; + }; + + /// Add nodes with only Create ACL + create("/A"); + create("/A/B"); + create("/A/C"); + create("/A/B/D"); + + { + int new_zxid = ++zxid; + + auto remove_request = std::make_shared(); + remove_request->path = "/A"; + remove_request->remove_nodes_limit = 4; + + storage.preprocessRequest(remove_request, 1, 0, new_zxid); + auto responses = storage.processRequest(remove_request, 1, new_zxid); + + EXPECT_EQ(responses.size(), 1); + EXPECT_EQ(responses[0].response->error, Coordination::Error::ZNOAUTH); + } +} + /// INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, /// CoordinationTest, /// ::testing::ValuesIn(std::initializer_list{CompressionParam{true, ".zstd"}, CompressionParam{false, ""}})); diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 79173503f28..18ee096569a 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -148,6 +148,7 @@ namespace DB M(Bool, storage_metadata_write_full_object_key, false, "Write disk metadata files with VERSION_FULL_OBJECT_KEY format", 0) \ M(UInt64, max_materialized_views_count_for_table, 0, "A limit on the number of materialized views attached to a table.", 0) \ M(UInt32, max_database_replicated_create_table_thread_pool_size, 1, "The number of threads to create tables during replica recovery in DatabaseReplicated. Zero means number of threads equal number of cores.", 0) \ + M(Bool, database_replicated_allow_detach_permanently, true, "Allow detaching tables permanently in Replicated databases", 0) \ M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \ M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \ M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \ @@ -169,6 +170,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ + M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \ M(Bool, disable_insertion_and_mutation, false, "Disable all insert/alter/delete queries. This setting will be enabled if someone needs read-only nodes to prevent insertion and mutation affect reading performance.", 0) /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 493752fc3fe..3a2c7b12b13 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -923,6 +923,9 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ + M(Int32, join_to_sort_minimum_perkey_rows, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys", 0) \ + M(Int32, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) \ + M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ @@ -945,6 +948,7 @@ class IColumn; M(Bool, parallel_replicas_prefer_local_join, 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.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as ' :: ' if archive has correct extension", 0) \ + M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 19f2d5ccdf0..6bd354ce05b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,6 +79,10 @@ static std::initializer_listgetServerSettings().database_replicated_allow_detach_permanently) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Support for DETACH TABLE PERMANENTLY is disabled"); + auto txn = local_context->getZooKeeperMetadataTransaction(); assert(!ddl_worker->isCurrentlyActive() || txn); if (txn && txn->isInitialQuery()) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index 0e133590b84..074a34bd083 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -124,7 +124,7 @@ public: std::string_view sqid = col_non_const->getDataAt(i).toView(); std::vector integers = sqids.decode(String(sqid)); res_nested_data.insert(integers.begin(), integers.end()); - res_offsets_data.push_back(integers.size()); + res_offsets_data.push_back(res_offsets_data.back() + integers.size()); } } else diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d04a73e384e..771c6a89caa 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include namespace DB @@ -38,6 +39,7 @@ namespace ErrorCodes extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; extern const int UNEXPECTED_CLUSTER; + extern const int INCONSISTENT_CLUSTER_DEFINITION; } namespace ClusterProxy @@ -439,7 +441,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr analyzed_read_from_merge_tree) { auto logger = getLogger("executeQueryWithParallelReplicas"); LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas", @@ -485,12 +488,12 @@ void executeQueryWithParallelReplicas( shard_num = column->getUInt(0); } - const auto shard_count = not_optimized_cluster->getShardCount(); ClusterPtr new_cluster = not_optimized_cluster; /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard /// shards are numbered in order of appearance in the cluster config if (shard_num > 0) { + const auto shard_count = not_optimized_cluster->getShardCount(); if (shard_num > shard_count) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -516,21 +519,147 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - auto external_tables = new_context->getExternalTables(); - auto read_from_remote = std::make_unique( - query_ast, - new_cluster, - storage_id, - header, - processed_stage, - new_context, - getThrottler(new_context), - std::move(scalars), - std::move(external_tables), - getLogger("ReadFromParallelRemoteReplicasStep"), - std::move(storage_limits)); + const auto & shard = new_cluster->getShardsInfo().at(0); + size_t max_replicas_to_use = settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) + { + LOG_INFO( + getLogger("ReadFromParallelRemoteReplicasStep"), + "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " + "Will use the latter number to execute the query.", + settings.max_parallel_replicas, + shard.getAllNodeCount()); + max_replicas_to_use = shard.getAllNodeCount(); + } - query_plan.addStep(std::move(read_from_remote)); + auto coordinator = std::make_shared(max_replicas_to_use, settings.parallel_replicas_mark_segment_size); + + auto external_tables = new_context->getExternalTables(); + + std::vector shuffled_pool; + if (max_replicas_to_use < shard.getAllNodeCount()) + { + // will be shuffled according to `load_balancing` setting + shuffled_pool = shard.pool->getShuffledPools(settings); + } + else + { + /// If all replicas in cluster are used for query execution, + /// try to preserve replicas order as in cluster definition. + /// It's important for data locality during query execution + /// independently of the query initiator + auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; + shuffled_pool = shard.pool->getShuffledPools(settings, priority_func); + } + + std::vector pools_to_use; + pools_to_use.reserve(shuffled_pool.size()); + for (auto & pool : shuffled_pool) + pools_to_use.emplace_back(std::move(pool.pool)); + + /// do not build local plan for distributed queries for now (address it later) + if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) + { + /// find local replica index in pool + std::optional local_replica_index; + for (size_t i = 0, s = pools_to_use.size(); i < s; ++i) + { + const auto & hostname = pools_to_use[i]->getHost(); + const auto found = std::find_if( + begin(shard.local_addresses), + end(shard.local_addresses), + [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); + if (found != shard.local_addresses.end()) + { + local_replica_index = i; + break; + } + } + if (!local_replica_index) + throw Exception( + ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting", + new_cluster->getName()); + + // resize the pool but keep local replicas in it (and update its index) + chassert(max_replicas_to_use <= pools_to_use.size()); + if (local_replica_index >= max_replicas_to_use) + { + std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]); + local_replica_index = max_replicas_to_use - 1; + } + pools_to_use.resize(max_replicas_to_use); + + auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas( + query_ast, + header, + new_context, + processed_stage, + coordinator, + std::move(analyzed_read_from_merge_tree), + local_replica_index.value()); + + if (!with_parallel_replicas) + { + query_plan = std::move(*local_plan); + return; + } + + LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value()); + + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + coordinator, + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + std::move(pools_to_use), + local_replica_index); + + auto remote_plan = std::make_unique(); + remote_plan->addStep(std::move(read_from_remote)); + + DataStreams input_streams; + input_streams.reserve(2); + input_streams.emplace_back(local_plan->getCurrentDataStream()); + input_streams.emplace_back(remote_plan->getCurrentDataStream()); + + std::vector plans; + plans.emplace_back(std::move(local_plan)); + plans.emplace_back(std::move(remote_plan)); + + auto union_step = std::make_unique(std::move(input_streams)); + query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + else + { + chassert(max_replicas_to_use <= pools_to_use.size()); + pools_to_use.resize(max_replicas_to_use); + + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + std::move(coordinator), + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + std::move(pools_to_use)); + + query_plan.addStep(std::move(read_from_remote)); + } } void executeQueryWithParallelReplicas( @@ -540,7 +669,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr analyzed_read_from_merge_tree) { QueryTreeNodePtr modified_query_tree = query_tree->clone(); rewriteJoinToGlobalJoin(modified_query_tree, context); @@ -550,7 +680,8 @@ void executeQueryWithParallelReplicas( = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); + executeQueryWithParallelReplicas( + query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree)); } void executeQueryWithParallelReplicas( @@ -646,6 +777,49 @@ void executeQueryWithParallelReplicasCustomKey( context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr); executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context); } + +bool canUseParallelReplicasOnInitiator(const ContextPtr & context) +{ + if (!context->canUseParallelReplicasOnInitiator()) + return false; + + auto cluster = context->getClusterForParallelReplicas(); + if (cluster->getShardCount() == 1) + return cluster->getShardsInfo()[0].getAllNodeCount() > 1; + + /// parallel replicas with distributed table + auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified + const auto it = scalars.find("_shard_num"); + if (it != scalars.end()) + { + const Block & block = it->second; + const auto & column = block.safeGetByPosition(0).column; + shard_num = column->getUInt(0); + } + if (shard_num > 0) + { + const auto shard_count = cluster->getShardCount(); + if (shard_num > shard_count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Shard number is greater than shard count: shard_num={} shard_count={} cluster={}", + shard_num, + shard_count, + cluster->getName()); + + return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1; + } + + if (cluster->getShardCount() > 1) + throw DB::Exception( + ErrorCodes::UNEXPECTED_CLUSTER, + "`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard", + cluster->getShardCount()); + + return false; +} + } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index c22fcd24f03..2a21f3e8255 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -35,6 +35,9 @@ using QueryTreeNodePtr = std::shared_ptr; class PlannerContext; using PlannerContextPtr = std::shared_ptr; +class IQueryPlanStep; +using QueryPlanStepPtr = std::unique_ptr; + namespace ClusterProxy { @@ -55,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function; AdditionalShardFilterGenerator getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns); +bool canUseParallelReplicasOnInitiator(const ContextPtr & context); + /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. /// `stream_factory` object encapsulates the logic of creating plans for a different type of query /// (currently SELECT, DESCRIBE). @@ -81,7 +86,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree = nullptr); void executeQueryWithParallelReplicas( QueryPlan & query_plan, @@ -98,7 +104,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree); void executeQueryWithParallelReplicasCustomKey( QueryPlan & query_plan, diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 373cc91ebcb..7adfb42fb51 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index fb5337158ba..858b4a78430 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -152,6 +152,7 @@ class ServerType; template class MergeTreeBackgroundExecutor; class AsyncLoader; +struct ICgroupsReader; struct TemporaryTableHolder; using TemporaryTablesMapping = std::map>; diff --git a/src/Interpreters/HashJoin/AddedColumns.cpp b/src/Interpreters/HashJoin/AddedColumns.cpp index 21cb6e401ed..2e1ecb0da72 100644 --- a/src/Interpreters/HashJoin/AddedColumns.cpp +++ b/src/Interpreters/HashJoin/AddedColumns.cpp @@ -34,6 +34,23 @@ void AddedColumns::buildOutput() { if (join_data_avg_perkey_rows < output_by_row_list_threshold) buildOutputFromBlocks(); + else if (join_data_sorted) + { + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) + { + if (row_ref_i) + { + const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); + col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows); + } + else + type_name[i].type->insertDefaultInto(*col); + } + } + } else { for (size_t i = 0; i < this->size(); ++i) diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index f1b95a63be6..4603d493329 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -66,6 +66,9 @@ public: , join_on_keys(join_on_keys_) , additional_filter_expression(additional_filter_expression_) , rows_to_add(left_block.rows()) + , join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows()) + , output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold()) + , join_data_sorted(join.getJoinedData()->sorted) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -113,8 +116,6 @@ public: if (columns[j]->isNullable() && !saved_column->isNullable()) nullable_column_ptrs[j] = typeid_cast(columns[j].get()); } - join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows(); - output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold(); } size_t size() const { return columns.size(); } @@ -149,6 +150,7 @@ public: bool output_by_row_list = false; size_t join_data_avg_perkey_rows = 0; size_t output_by_row_list_threshold = 0; + bool join_data_sorted = false; IColumn::Filter filter; void reserve(bool need_replicate) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 9c07a71e614..1b8b45b94ea 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -649,7 +649,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) } data->keys_to_join = total_rows; shrinkStoredBlocksToFit(total_bytes); - return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } @@ -1361,4 +1360,96 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table return false; } +template +void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) +{ + constexpr JoinFeatures join_features; + if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged."); + else + { + auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) + { + auto it = rows_ref.begin(); + if (it.ok()) + { + if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE) + blocks.emplace_back(it->block->cloneEmpty()); + } + else + { + return; + } + auto & block = blocks.back(); + size_t start_row = block.rows(); + for (; it.ok(); ++it) + { + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = block.getByPosition(i).column->assumeMutableRef(); + col.insertFrom(*it->block->getByPosition(i).column, it->row_num); + } + } + if (block.rows() > start_row) + { + RowRefList new_rows_ref(&block, start_row, block.rows() - start_row); + rows_ref = std::move(new_rows_ref); + } + }; + + auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map) + { + switch (data->type) + { + #define M(TYPE) \ + case Type::TYPE: \ + {\ + rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \ + break; \ + } + APPLY_FOR_JOIN_VARIANTS(M) + #undef M + default: + break; + } + }; + BlocksList sorted_blocks; + visit_rows_map(sorted_blocks, map); + data->blocks.swap(sorted_blocks); + } +} + +void HashJoin::tryRerangeRightTableData() +{ + if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All) + return; + + /// We should not rerange the right table on such conditions: + /// 1. the right table is already reranged by key or it is empty. + /// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys. + /// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation. + /// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key. + if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightMaximumTableRows() || data->avgPerKeyRows() < table_join->sortRightMinimumPerkeyRows()) + return; + + if (data->keys_to_join == 0) + data->keys_to_join = getTotalRowCount(); + + /// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom` + /// to replace column's `insertFrom` to make the output. + if (sample_block_with_columns_to_add.columns() == 0) + { + LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join); + return; + } + [[maybe_unused]] bool result = joinDispatch( + kind, + strictness, + data->maps.front(), + /*prefer_use_maps_all*/ false, + [&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl(map_); }); + chassert(result); + data->sorted = true; +} + } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index d645b8e9273..230343691ea 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -345,11 +345,12 @@ public: size_t blocks_allocated_size = 0; size_t blocks_nullmaps_allocated_size = 0; - /// Number of rows of right table to join size_t rows_to_join = 0; /// Number of keys of right table to join size_t keys_to_join = 0; + /// Whether the right table reranged by key + bool sorted = false; size_t avgPerKeyRows() const { @@ -465,6 +466,10 @@ private: void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; + + void tryRerangeRightTableData() override; + template + void tryRerangeRightTableDataImpl(Map & map); }; } diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 97ad57d26ea..c5b54a62f36 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -83,7 +83,6 @@ public: const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get = false); - private: template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); @@ -199,4 +198,3 @@ extern template class HashJoinMethods; extern template class HashJoinMethods; } - diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 7374348da50..8f648de2538 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -115,6 +115,7 @@ public: /// Peek next stream of delayed joined blocks. virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; } virtual bool hasDelayedBlocks() const { return false; } + virtual void tryRerangeRightTableData() {} virtual IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6b79552ef0b..60d4abd0ef8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1065,6 +1065,11 @@ namespace void setNullTableEngine(ASTStorage & storage) { + storage.forEachPointerToChild([](void ** ptr) mutable + { + *ptr = nullptr; + }); + auto engine_ast = std::make_shared(); engine_ast->name = "Null"; engine_ast->no_empty_args = true; @@ -1151,7 +1156,9 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const else if (getContext()->getSettingsRef().restore_replace_external_engines_to_null) { if (StorageFactory::instance().getStorageFeatures(create.storage->engine->name).source_access_type != AccessType::NONE) + { setNullTableEngine(*create.storage); + } } return; } diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 11df92d071f..916dee01431 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1224,9 +1224,18 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, - plan, *data, storage_snapshot, - part, alter_conversions, required_columns, - apply_deleted_mask_, std::move(filter), context_, + plan, + *data, + storage_snapshot, + part, + alter_conversions, + required_columns, + nullptr, + apply_deleted_mask_, + std::move(filter), + false, + false, + context_, getLogger("MutationsInterpreter")); } else diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 7c98c47dd11..f8ac68191d6 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -123,6 +123,7 @@ struct RowRefList : RowRef RowRefList() {} /// NOLINT RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {} + RowRefList(const Block * block_, size_t row_start_, size_t rows_) : RowRef(block_, row_start_), rows(static_cast(rows_)) {} ForwardIterator begin() const { return ForwardIterator(this); } diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 872a9f864df..079029695c9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -55,9 +55,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index e3c83dc748e..5fab419a32b 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -14,7 +14,10 @@ public: ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~ServerAsynchronousMetrics() override; private: diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 95b76c60063..c656a1a797b 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk) Chunk Squashing::add(Chunk && input_chunk) { - if (!input_chunk) + if (!input_chunk || input_chunk.getNumRows() == 0) return {}; /// Just read block is already enough. diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 138085f0710..59a0374051f 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -116,6 +116,9 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) + , sort_right_minimum_perkey_rows(settings.join_to_sort_minimum_perkey_rows) + , sort_right_maximum_table_rows(settings.join_to_sort_maximum_table_rows) + , allow_join_sorting(settings.allow_experimental_join_right_table_sorting) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4d626084d81..e1bae55a4ed 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -149,6 +149,9 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; const size_t output_by_rowlist_perkey_rows_threshold = 0; + const size_t sort_right_minimum_perkey_rows = 0; + const size_t sort_right_maximum_table_rows = 0; + const bool allow_join_sorting = false; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. size_t max_memory_usage = 0; @@ -297,6 +300,9 @@ public: } size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } + size_t sortRightMinimumPerkeyRows() const { return sort_right_minimum_perkey_rows; } + size_t sortRightMaximumTableRows() const { return sort_right_maximum_table_rows; } + bool allowJoinSorting() const { return allow_join_sorting; } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } diff --git a/src/Interpreters/ZooKeeperLog.cpp b/src/Interpreters/ZooKeeperLog.cpp index 0d3063a569e..769757a5fba 100644 --- a/src/Interpreters/ZooKeeperLog.cpp +++ b/src/Interpreters/ZooKeeperLog.cpp @@ -93,6 +93,7 @@ ColumnsDescription ZooKeeperLogElement::getColumnsDescription() {"FilteredList", static_cast(Coordination::OpNum::FilteredList)}, {"CheckNotExists", static_cast(Coordination::OpNum::CheckNotExists)}, {"CreateIfNotExists", static_cast(Coordination::OpNum::CreateIfNotExists)}, + {"RemoveRecursive", static_cast(Coordination::OpNum::RemoveRecursive)}, }); auto error_enum = getCoordinationErrorCodesEnumType(); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6c22e71bccf..be9423852c1 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1118,6 +1118,19 @@ static std::tuple executeQueryImpl( && settings.use_query_cache && !internal && client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY + /// Bug 67476: Avoid that the query cache stores truncated results if the query ran with a non-THROW overflow mode and hit a limit. + /// This is more workaround than a fix ... unfortunately it is hard to detect from the perspective of the query cache that the + /// query result is truncated. + && (settings.read_overflow_mode == OverflowMode::THROW + && settings.read_overflow_mode_leaf == OverflowMode::THROW + && settings.group_by_overflow_mode == OverflowMode::THROW + && settings.sort_overflow_mode == OverflowMode::THROW + && settings.result_overflow_mode == OverflowMode::THROW + && settings.timeout_overflow_mode == OverflowMode::THROW + && settings.set_overflow_mode == OverflowMode::THROW + && settings.join_overflow_mode == OverflowMode::THROW + && settings.transfer_overflow_mode == OverflowMode::THROW + && settings.distinct_overflow_mode == OverflowMode::THROW) && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index bc31af32a20..76583c82583 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -893,31 +893,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres /// query_plan can be empty if there is nothing to read if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings)) { - // (1) find read step - QueryPlan::Node * node = query_plan.getRootNode(); - ReadFromMergeTree * reading = nullptr; - while (node) - { - reading = typeid_cast(node->step.get()); - if (reading) - break; - - QueryPlan::Node * prev_node = node; - if (!node->children.empty()) - { - chassert(node->children.size() == 1); - node = node->children.at(0); - } - else - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Step is expected to be ReadFromMergeTree but it's {}", - prev_node->step->getName()); - } - } - - chassert(reading); if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) { if (auto cluster = query_context->getClusterForParallelReplicas(); @@ -940,21 +915,50 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan = std::move(query_plan_parallel_replicas); } } - else if (query_context->canUseParallelReplicasOnInitiator()) + else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context)) { + // (1) find read step + QueryPlan::Node * node = query_plan.getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + reading = typeid_cast(node->step.get()); + if (reading) + break; + + QueryPlan::Node * prev_node = node; + if (!node->children.empty()) + { + chassert(node->children.size() == 1); + node = node->children.at(0); + } + else + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Step is expected to be ReadFromMergeTree but it's {}", + prev_node->step->getName()); + } + } + + chassert(reading); + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { auto result_ptr = reading->selectRangesToRead(); - UInt64 rows_to_read = result_ptr->selected_rows; + + reading->setAnalyzedResult(std::move(result_ptr)); + if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read) rows_to_read = table_expression_query_info.trivial_limit; if (max_block_size_limited && (max_block_size_limited < rows_to_read)) rows_to_read = max_block_size_limited; - const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; + const size_t number_of_replicas_to_use + = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica; LOG_TRACE( getLogger("Planner"), "Estimated {} rows to read. It is enough work for {} parallel replicas", @@ -980,6 +984,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { from_stage = QueryProcessingStage::WithMergeableState; QueryPlan query_plan_parallel_replicas; + QueryPlanStepPtr reading_step = std::move(node->step); ClusterProxy::executeQueryWithParallelReplicas( query_plan_parallel_replicas, storage->getStorageID(), @@ -987,9 +992,24 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression_query_info.query_tree, table_expression_query_info.planner_context, query_context, - table_expression_query_info.storage_limits); + table_expression_query_info.storage_limits, + std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); } + else + { + QueryPlan query_plan_no_parallel_replicas; + storage->read( + query_plan_no_parallel_replicas, + columns_names, + storage_snapshot, + table_expression_query_info, + query_context, + from_stage, + max_block_size, + max_streams); + query_plan = std::move(query_plan_no_parallel_replicas); + } } } diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 39edb1e6516..25481d06670 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -52,7 +52,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & storage = table_node.getStorage(); /// Here we check StorageDummy as well, to support a query tree with replaced storages. if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) + { + /// parallel replicas is not supported with FINAL + if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal()) + return {}; + return res; + } return {}; } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index 3b5e4e06953..d99f9a7d1f1 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -12,12 +12,15 @@ namespace DB struct Settings; class QueryStatus; using QueryStatusPtr = std::shared_ptr; +struct ITemporaryFileLookup; +using TemporaryFileLookupPtr = std::shared_ptr; struct BuildQueryPipelineSettings { ExpressionActionsSettings actions_settings; QueryStatusPtr process_list_element; ProgressCallback progress_callback = nullptr; + TemporaryFileLookupPtr temporary_file_lookup; const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } static BuildQueryPipelineSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/ConvertingActions.cpp b/src/Processors/QueryPlan/ConvertingActions.cpp new file mode 100644 index 00000000000..b9703ef59cd --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.cpp @@ -0,0 +1,32 @@ +#include +#include +#include + +namespace DB +{ + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) +{ + if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) + return; + + auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; + + auto get_converting_dag = [mode](const Block & block_, const Block & header_) + { + /// Convert header structure to expected. + /// Also we ignore constants from result and replace it with constants from header. + /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. + return ActionsDAG::makeConvertingActions( + block_.getColumnsWithTypeAndName(), + header_.getColumnsWithTypeAndName(), + mode, + true); + }; + + auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); + auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); + plan.addStep(std::move(converting)); +} + +} diff --git a/src/Processors/QueryPlan/ConvertingActions.h b/src/Processors/QueryPlan/ConvertingActions.h new file mode 100644 index 00000000000..6bdf9b8af9a --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ +class QueryPlan; +class Block; + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects); +} diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index d8624a1c99b..eb699858bdf 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -2,43 +2,13 @@ #include #include -#include #include #include -#include +#include namespace DB { -namespace -{ - -void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) -{ - if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) - return; - - auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; - - auto get_converting_dag = [mode](const Block & block_, const Block & header_) - { - /// Convert header structure to expected. - /// Also we ignore constants from result and replace it with constants from header. - /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. - return ActionsDAG::makeConvertingActions( - block_.getColumnsWithTypeAndName(), - header_.getColumnsWithTypeAndName(), - mode, - true); - }; - - auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); - plan.addStep(std::move(converting)); -} - -} - std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 9dc4992cddd..ac7fcdcf83f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl return {}; } +static bool readingFromParallelReplicas(const QueryPlan::Node * node) +{ + IQueryPlanStep * step = node->step.get(); + while (!node->children.empty()) + { + step = node->children.front()->step.get(); + node = node->children.front(); + } + + return typeid_cast(step); +} + void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) std::vector infos; infos.reserve(node.children.size()); + for (const auto * child : union_node->children) + { + /// in case of parallel replicas + /// avoid applying read-in-order optimization for local replica + /// since it will lead to different parallel replicas modes + /// between local and remote nodes + if (readingFromParallelReplicas(child)) + return; + } + for (auto * child : union_node->children) { infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update)); diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp new file mode 100644 index 00000000000..050044edd3a --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -0,0 +1,97 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +std::pair, bool> createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr analyzed_read_from_merge_tree, + size_t replica_number) +{ + checkStackSize(); + + auto query_plan = std::make_unique(); + auto new_context = Context::createCopy(context); + + /// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter. + if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) + processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + + /// Do not apply AST optimizations, because query + /// is already optimized and some optimizations + /// can be applied only for non-distributed tables + /// and we can produce query, inconsistent with remote plans. + auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations(); + + /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to + /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace + /// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289). + new_context->setSetting("enable_positional_arguments", Field(false)); + new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); + query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); + + QueryPlan::Node * node = query_plan->getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + reading = typeid_cast(node->step.get()); + if (reading) + break; + + if (!node->children.empty()) + node = node->children.at(0); + else + node = nullptr; + } + + if (!reading) + /// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource + return {std::move(query_plan), false}; + + ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr; + if (analyzed_read_from_merge_tree.get()) + { + auto * analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); + if (analyzed_merge_tree) + analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult(); + } + + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) + { coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); }; + + MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional + { return coordinator->handleRequest(std::move(req)); }; + + auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep( + analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number); + node->step = std::move(read_from_merge_tree_parallel_replicas); + + addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); + + return {std::move(query_plan), true}; +} + +} diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h new file mode 100644 index 00000000000..ab0bbeaeeff --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +std::pair, bool> createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr read_from_merge_tree, + size_t replica_number); +} diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 339d97126f4..218f0a61a48 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -277,7 +277,10 @@ ReadFromMergeTree::ReadFromMergeTree( std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading) + bool enable_parallel_reading_, + std::optional all_ranges_callback_, + std::optional read_task_callback_, + std::optional number_of_current_replica_) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(all_column_names_), query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) @@ -295,13 +298,21 @@ ReadFromMergeTree::ReadFromMergeTree( , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) - , is_parallel_reading_from_replicas(enable_parallel_reading) + , is_parallel_reading_from_replicas(enable_parallel_reading_) , enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization) + , number_of_current_replica(number_of_current_replica_) { if (is_parallel_reading_from_replicas) { - all_ranges_callback = context->getMergeTreeAllRangesCallback(); - read_task_callback = context->getMergeTreeReadTaskCallback(); + if (all_ranges_callback_.has_value()) + all_ranges_callback = all_ranges_callback_.value(); + else + all_ranges_callback = context->getMergeTreeAllRangesCallback(); + + if (read_task_callback_.has_value()) + read_task_callback = read_task_callback_.value(); + else + read_task_callback = context->getMergeTreeReadTaskCallback(); } const auto & settings = context->getSettingsRef(); @@ -335,11 +346,33 @@ ReadFromMergeTree::ReadFromMergeTree( enable_vertical_final); } +std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( + AnalysisResultPtr analyzed_result_ptr_, + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number) +{ + const bool enable_parallel_reading = true; + return std::make_unique( + prepared_parts, + mutations_snapshot, + all_column_names, + data, + getQueryInfo(), + getStorageSnapshot(), + getContext(), + block_size.max_block_size_rows, + requested_num_streams, + max_block_numbers_to_read, + log, + std::move(analyzed_result_ptr_), + enable_parallel_reading, + all_ranges_callback_, + read_task_callback_, + replica_number); +} -Pipe ReadFromMergeTree::readFromPoolParallelReplicas( - RangesInDataParts parts_with_range, - Names required_columns, - PoolSettings pool_settings) +Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) { const auto & client_info = context->getClientInfo(); @@ -347,7 +380,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas( { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), - .number_of_current_replica = client_info.number_of_current_replica, + .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), }; /// We have a special logic for local replica. It has to read less data, because in some cases it should @@ -529,7 +562,7 @@ Pipe ReadFromMergeTree::readInOrder( { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), - .number_of_current_replica = client_info.number_of_current_replica, + .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), }; auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; @@ -584,11 +617,12 @@ Pipe ReadFromMergeTree::readInOrder( context); } - /// Actually it means that parallel reading from replicas enabled - /// and we have to collaborate with initiator. - /// In this case we won't set approximate rows, because it will be accounted multiple times. - const auto in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; - const bool set_total_rows_approx = !is_parallel_reading_from_replicas; + /// If parallel replicas enabled, set total rows in progress here only on initiator with local plan + /// Otherwise rows will counted multiple times + const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; + const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas + && context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator(); + const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator; Pipes pipes; for (size_t i = 0; i < parts_with_ranges.size(); ++i) @@ -1422,11 +1456,8 @@ static void buildIndexes( const auto & settings = context->getSettingsRef(); - indexes.emplace(ReadFromMergeTree::Indexes{{ - filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression}, {}, {}, {}, {}, false, {}}); + indexes.emplace( + ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}}); if (metadata_snapshot->hasPartitionKey()) { @@ -1978,6 +2009,33 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons { auto result = getAnalysisResult(); + if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator() + && context->getSettingsRef().parallel_replicas_local_plan) + { + CoordinationMode mode = CoordinationMode::Default; + switch (result.read_type) + { + case ReadFromMergeTree::ReadType::Default: + mode = CoordinationMode::Default; + break; + case ReadFromMergeTree::ReadType::InOrder: + mode = CoordinationMode::WithOrder; + break; + case ReadFromMergeTree::ReadType::InReverseOrder: + mode = CoordinationMode::ReverseOrder; + break; + case ReadFromMergeTree::ReadType::ParallelReplicas: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read type can't be ParallelReplicas on initiator"); + } + + chassert(number_of_current_replica.has_value()); + chassert(all_ranges_callback.has_value()); + + /// initialize working set from local replica + all_ranges_callback.value()( + InitialAllRangesAnnouncement(mode, result.parts_with_ranges.getDescriptions(), number_of_current_replica.value())); + } + if (enable_remove_parts_from_snapshot_optimization) { /// Do not keep data parts in snapshot. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 0bdf0427dd0..60707c406b3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -121,7 +121,16 @@ public: std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading); + bool enable_parallel_reading_, + std::optional all_ranges_callback_ = std::nullopt, + std::optional read_task_callback_ = std::nullopt, + std::optional number_of_current_replica_ = std::nullopt); + + std::unique_ptr createLocalParallelReplicasReadingStep( + AnalysisResultPtr analyzed_result_ptr_, + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } @@ -143,6 +152,11 @@ public: struct Indexes { + explicit Indexes(KeyCondition key_condition_) + : key_condition(std::move(key_condition_)) + , use_skip_indexes(false) + {} + KeyCondition key_condition; std::optional partition_pruner; std::optional minmax_idx_condition; @@ -268,6 +282,7 @@ private: std::optional read_task_callback; bool enable_vertical_final = false; bool enable_remove_parts_from_snapshot_optimization = true; + std::optional number_of_current_replica; }; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cf11052cd59..3df46eb1987 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -21,8 +21,8 @@ #include #include #include -#include -#include + +#include namespace DB { @@ -362,6 +362,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, + ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, @@ -369,11 +370,14 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_) + std::shared_ptr storage_limits_, + std::vector pools_to_use_, + std::optional exclude_pool_index_) : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) , query_ast(query_ast_) , storage_id(storage_id_) + , coordinator(std::move(coordinator_)) , stage(std::move(stage_)) , context(context_) , throttler(throttler_) @@ -381,16 +385,24 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , external_tables{external_tables_} , storage_limits(std::move(storage_limits_)) , log(log_) + , pools_to_use(std::move(pools_to_use_)) + , exclude_pool_index(exclude_pool_index_) { chassert(cluster->getShardCount() == 1); - std::vector description; - description.push_back(fmt::format("query: {}", formattedAST(query_ast))); + std::vector replicas; + replicas.reserve(pools_to_use.size()); - for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) - description.push_back(fmt::format("Replica: {}", pool->getHost())); + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) + { + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; - setStepDescription(boost::algorithm::join(description, ", ")); + replicas.push_back(pools_to_use[i]->getAddress()); + } + + auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", ")); + setStepDescription(std::move(description)); } void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description) @@ -406,48 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder() void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; - const Settings & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - const auto & shard = cluster->getShardsInfo().at(0); - size_t max_replicas_to_use = current_settings.max_parallel_replicas; - if (max_replicas_to_use > shard.getAllNodeCount()) + std::vector addresses; + addresses.reserve(pools_to_use.size()); + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { - LOG_INFO( - getLogger("ReadFromParallelRemoteReplicasStep"), - "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " - "Will use the latter number to execute the query.", - current_settings.max_parallel_replicas, - shard.getAllNodeCount()); - max_replicas_to_use = shard.getAllNodeCount(); + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; + + addresses.emplace_back(pools_to_use[i]->getAddress()); } + LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", ")); - std::vector shuffled_pool; - if (max_replicas_to_use < shard.getAllNodeCount()) + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { - shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(max_replicas_to_use); - } - else - { - /// try to preserve replicas order if all replicas in cluster are used for query execution - /// it's important for data locality during query execution - auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; - shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); - } + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; - coordinator - = std::make_shared(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size); - - for (size_t i=0; i < max_replicas_to_use; ++i) - { - IConnections::ReplicaInfo replica_info - { + IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, }; - addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info); + addPipeForSingeReplica(pipes, pools_to_use[i], replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 1adb26b2915..74389c8f9eb 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -70,6 +70,7 @@ public: ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, + ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, @@ -77,7 +78,9 @@ public: Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_); + std::shared_ptr storage_limits_, + std::vector pools_to_use, + std::optional exclude_pool_index_ = std::nullopt); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -100,6 +103,8 @@ private: Tables external_tables; std::shared_ptr storage_limits; LoggerPtr log; + std::vector pools_to_use; + std::optional exclude_pool_index; }; } diff --git a/src/Processors/QueryPlan/TemporaryFiles.h b/src/Processors/QueryPlan/TemporaryFiles.h new file mode 100644 index 00000000000..943a2c2b0a4 --- /dev/null +++ b/src/Processors/QueryPlan/TemporaryFiles.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class WriteBuffer; +class ReadBuffer; + +/// Interface for accessing temporary files by some logical name (or id). +/// While building query pipeline processors can lookup temporary files by some id and use them for writing and/or reading temporary data +/// without knowing what exactly is behind the name: local file, memory buffer, object in cloud storage, etc. +struct ITemporaryFileLookup : boost::noncopyable +{ + virtual ~ITemporaryFileLookup() = default; + + /// Give the caller a temporary write buffer, but don't give away the ownership. + virtual WriteBuffer & getTemporaryFileForWriting(const String & file_id) = 0; + + /// Give the caller a temporary read buffer, it exclusively belongs to the caller. + /// Other callers can get their own read buffer for the same temporary file. + virtual std::unique_ptr getTemporaryFileForReading(const String & file_id) = 0; +}; + +using TemporaryFileLookupPtr = std::shared_ptr; + +} diff --git a/src/Processors/Transforms/ColumnGathererTransform.cpp b/src/Processors/Transforms/ColumnGathererTransform.cpp index 52fa42fdb51..f266d5c2e2f 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.cpp +++ b/src/Processors/Transforms/ColumnGathererTransform.cpp @@ -183,13 +183,14 @@ void ColumnGathererStream::consume(Input & input, size_t source_num) ColumnGathererTransform::ColumnGathererTransform( const Block & header, size_t num_inputs, - ReadBuffer & row_sources_buf_, + std::unique_ptr row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, bool is_result_sparse_) : IMergingTransform( num_inputs, header, header, /*have_all_inputs_=*/ true, /*limit_hint_=*/ 0, /*always_read_till_end_=*/ false, - num_inputs, row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_) + num_inputs, *row_sources_buf_, block_preferred_size_rows_, block_preferred_size_bytes_, is_result_sparse_) + , row_sources_buf_holder(std::move(row_sources_buf_)) , log(getLogger("ColumnGathererStream")) { if (header.columns() != 1) diff --git a/src/Processors/Transforms/ColumnGathererTransform.h b/src/Processors/Transforms/ColumnGathererTransform.h index fbc9a6bfcc6..ce2671ce0bf 100644 --- a/src/Processors/Transforms/ColumnGathererTransform.h +++ b/src/Processors/Transforms/ColumnGathererTransform.h @@ -115,7 +115,7 @@ public: ColumnGathererTransform( const Block & header, size_t num_inputs, - ReadBuffer & row_sources_buf_, + std::unique_ptr row_sources_buf_, size_t block_preferred_size_rows_, size_t block_preferred_size_bytes_, bool is_result_sparse_); @@ -124,6 +124,8 @@ public: protected: void onFinish() override; + + std::unique_ptr row_sources_buf_holder; /// Keep ownership of row_sources_buf while it's in use by ColumnGathererStream. LoggerPtr log; }; diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index ca204bcb482..f2fb6327129 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -299,13 +299,17 @@ IProcessor::Status FillingRightJoinSideTransform::prepare() void FillingRightJoinSideTransform::work() { - auto block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); + auto & input = inputs.front(); + auto block = input.getHeader().cloneWithColumns(chunk.detachColumns()); if (for_totals) join->setTotals(block); else stop_reading = !join->addBlockToJoin(block); + if (input.isFinished()) + join->tryRerangeRightTableData(); + set_totals = for_totals; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 2b9a7295198..8083f7da24d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -2133,7 +2133,7 @@ bool TCPHandler::receiveUnexpectedData(bool throw_exception) std::shared_ptr maybe_compressed_in; if (last_block_in.compression == Protocol::Compression::Enable) - maybe_compressed_in = std::make_shared(*in, /* allow_different_codecs */ true); + maybe_compressed_in = std::make_shared(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY); else maybe_compressed_in = in; @@ -2157,7 +2157,7 @@ void TCPHandler::initBlockInput() /// with another codec that the rest of the data. Example: data sent by Distributed tables. if (state.compression == Protocol::Compression::Enable) - state.maybe_compressed_in = std::make_shared(*in, /* allow_different_codecs */ true); + state.maybe_compressed_in = std::make_shared(*in, /* allow_different_codecs */ true, /* external_data */ query_kind != ClientInfo::QueryKind::SECONDARY_QUERY); else state.maybe_compressed_in = in; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 7365b9334aa..0beeffcb267 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -38,6 +38,12 @@ #include #include #include +#include +#include +#include +#include +#include +#include #include #include #include @@ -85,6 +91,68 @@ static ColumnsStatistics getStatisticsForColumns( return all_statistics; } +/// Manages the "rows_sources" temporary file that is used during vertical merge. +class RowsSourcesTemporaryFile : public ITemporaryFileLookup +{ +public: + /// A logical name of the temporary file under which it will be known to the plan steps that use it. + static constexpr auto FILE_ID = "rows_sources"; + + explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_) + : tmp_disk(std::make_unique(temporary_data_on_disk_)) + , uncompressed_write_buffer(tmp_disk->createRawStream()) + , tmp_file_name_on_disk(uncompressed_write_buffer->getFileName()) + { + } + + WriteBuffer & getTemporaryFileForWriting(const String & name) override + { + if (name != FILE_ID) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name); + + if (write_buffer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was already requested for writing, there musto be only one writer"); + + write_buffer = (std::make_unique(*uncompressed_write_buffer)); + return *write_buffer; + } + + std::unique_ptr getTemporaryFileForReading(const String & name) override + { + if (name != FILE_ID) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name); + + if (!finalized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file is not finalized yet"); + + /// tmp_disk might not create real file if no data was written to it. + if (final_size == 0) + return std::make_unique(); + + /// Reopen the file for each read so that multiple reads can be performed in parallel and there is no need to seek to the beginning. + auto raw_file_read_buffer = std::make_unique(tmp_file_name_on_disk); + return std::make_unique(std::move(raw_file_read_buffer)); + } + + /// Returns written data size in bytes + size_t finalizeWriting() + { + write_buffer->finalize(); + uncompressed_write_buffer->finalize(); + finalized = true; + final_size = write_buffer->count(); + return final_size; + } + +private: + std::unique_ptr tmp_disk; + std::unique_ptr uncompressed_write_buffer; + std::unique_ptr write_buffer; + const String tmp_file_name_on_disk; + bool finalized = false; + size_t final_size = 0; +}; + static void addMissedColumnsToSerializationInfos( size_t num_rows_in_parts, const Names & part_columns, @@ -120,19 +188,19 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu std::set key_columns(sort_key_columns_vec.cbegin(), sort_key_columns_vec.cend()); /// Force sign column for Collapsing mode - if (ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing) - key_columns.emplace(ctx->merging_params.sign_column); + if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing) + key_columns.emplace(global_ctx->merging_params.sign_column); /// Force version column for Replacing mode - if (ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing) + if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing) { - key_columns.emplace(ctx->merging_params.is_deleted_column); - key_columns.emplace(ctx->merging_params.version_column); + key_columns.emplace(global_ctx->merging_params.is_deleted_column); + key_columns.emplace(global_ctx->merging_params.version_column); } /// Force sign column for VersionedCollapsing mode. Version is already in primary key. - if (ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) - key_columns.emplace(ctx->merging_params.sign_column); + if (global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing) + key_columns.emplace(global_ctx->merging_params.sign_column); /// Force to merge at least one column in case of empty key if (key_columns.empty()) @@ -190,7 +258,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu } } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const { ProfileEvents::increment(ProfileEvents::Merge); @@ -201,7 +269,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() // E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge. local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_"; } - const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : ""; + + const String local_tmp_suffix = global_ctx->parent_part ? global_ctx->suffix : ""; if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed)) throw Exception(ErrorCodes::ABORTED, "Cancelled merging parts"); @@ -226,7 +295,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() LOG_DEBUG(ctx->log, "DEDUPLICATE BY ('{}')", fmt::join(global_ctx->deduplicate_by_columns, "', '")); } - ctx->disk = global_ctx->space_reservation->getDisk(); + global_ctx->disk = global_ctx->space_reservation->getDisk(); auto local_tmp_part_basename = local_tmp_prefix + global_ctx->future_part->name + local_tmp_suffix; std::optional builder; @@ -238,7 +307,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } else { - auto local_single_disk_volume = std::make_shared("volume_" + global_ctx->future_part->name, ctx->disk, 0); + auto local_single_disk_volume = std::make_shared("volume_" + global_ctx->future_part->name, global_ctx->disk, 0); builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename)); builder->withPartStorageType(global_ctx->future_part->part_format.storage_type); } @@ -359,8 +428,6 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->compression_codec = global_ctx->data->getCompressionCodecForPart( global_ctx->merge_list_element_ptr->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge); - ctx->tmp_disk = std::make_unique(global_ctx->context->getTempDataOnDisk()); - switch (global_ctx->chosen_merge_algorithm) { case MergeAlgorithm::Horizontal: @@ -373,8 +440,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() } case MergeAlgorithm::Vertical: { - ctx->rows_sources_uncompressed_write_buf = ctx->tmp_disk->createRawStream(); - ctx->rows_sources_write_buf = std::make_unique(*ctx->rows_sources_uncompressed_write_buf); + ctx->rows_sources_temporary_file = std::make_shared(global_ctx->context->getTempDataOnDisk()); std::map local_merged_column_to_size; for (const auto & part : global_ctx->future_part->parts) @@ -494,11 +560,9 @@ MergeTask::StageRuntimeContextPtr MergeTask::ExecuteAndFinalizeHorizontalPart::g auto new_ctx = std::make_shared(); - new_ctx->rows_sources_write_buf = std::move(ctx->rows_sources_write_buf); - new_ctx->rows_sources_uncompressed_write_buf = std::move(ctx->rows_sources_uncompressed_write_buf); + new_ctx->rows_sources_temporary_file = std::move(ctx->rows_sources_temporary_file); new_ctx->column_sizes = std::move(ctx->column_sizes); new_ctx->compression_codec = std::move(ctx->compression_codec); - new_ctx->tmp_disk = std::move(ctx->tmp_disk); new_ctx->it_name_and_type = std::move(ctx->it_name_and_type); new_ctx->read_with_direct_io = std::move(ctx->read_with_direct_io); new_ctx->need_sync = std::move(ctx->need_sync); @@ -554,9 +618,9 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::prepareProjectionsToMergeAndRe const bool merge_may_reduce_rows = global_ctx->cleanup || global_ctx->deduplicate || - ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || - ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || - ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; const auto & projections = global_ctx->metadata_snapshot->getProjections(); @@ -664,7 +728,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() const { /// In case if there are no projections we didn't construct a task if (!ctx->merge_projection_parts_task_ptr) @@ -683,7 +747,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // N return true; } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); @@ -755,11 +819,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed); /// Ensure data has written to disk. - ctx->rows_sources_write_buf->finalize(); - ctx->rows_sources_uncompressed_write_buf->finalize(); - ctx->rows_sources_uncompressed_write_buf->finalize(); - - size_t rows_sources_count = ctx->rows_sources_write_buf->count(); + size_t rows_sources_count = ctx->rows_sources_temporary_file->finalizeWriting(); /// In special case, when there is only one source part, and no rows were skipped, we may have /// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total /// number of input rows. @@ -770,29 +830,6 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const "of bytes written to rows_sources file ({}). It is a bug.", sum_input_rows_exact, input_rows_filtered, rows_sources_count); - /// TemporaryDataOnDisk::createRawStream returns WriteBufferFromFile implementing IReadableWriteBuffer - /// and we expect to get ReadBufferFromFile here. - /// So, it's relatively safe to use dynamic_cast here and downcast to ReadBufferFromFile. - auto * wbuf_readable = dynamic_cast(ctx->rows_sources_uncompressed_write_buf.get()); - std::unique_ptr reread_buf = wbuf_readable ? wbuf_readable->tryGetReadBuffer() : nullptr; - if (!reread_buf) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot read temporary file {}", ctx->rows_sources_uncompressed_write_buf->getFileName()); - - auto * reread_buffer_raw = dynamic_cast(reread_buf.get()); - if (!reread_buffer_raw) - { - const auto & reread_buf_ref = *reread_buf; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected ReadBufferFromFileBase, but got {}", demangle(typeid(reread_buf_ref).name())); - } - /// Move ownership from std::unique_ptr to std::unique_ptr for CompressedReadBufferFromFile. - /// First, release ownership from unique_ptr to base type. - reread_buf.release(); /// NOLINT(bugprone-unused-return-value,hicpp-ignored-remove-result): we already have the pointer value in `reread_buffer_raw` - - /// Then, move ownership to unique_ptr to concrete type. - std::unique_ptr reread_buffer_from_file(reread_buffer_raw); - - /// CompressedReadBufferFromFile expects std::unique_ptr as argument. - ctx->rows_sources_read_buf = std::make_unique(std::move(reread_buffer_from_file)); ctx->it_name_and_type = global_ctx->gathering_columns.cbegin(); const auto & settings = global_ctx->context->getSettingsRef(); @@ -812,33 +849,158 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch; if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); + ctx->prepared_pipeline = createPipelineForReadingOneColumn(ctx->it_name_and_type->name); return false; } -Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & column_name) const +/// Gathers values from all parts for one column using rows sources temporary file +class ColumnGathererStep : public ITransformingStep { - Pipes pipes; +public: + ColumnGathererStep( + const DataStream & input_stream_, + const String & rows_sources_temporary_file_name_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool is_result_sparse_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + , rows_sources_temporary_file_name(rows_sources_temporary_file_name_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , is_result_sparse(is_result_sparse_) + {} + + String getName() const override { return "ColumnGatherer"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override + { + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + if (!pipeline_settings.temporary_file_lookup) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge"); + + auto rows_sources_read_buf = pipeline_settings.temporary_file_lookup->getTemporaryFileForReading(rows_sources_temporary_file_name); + + auto transform = std::make_unique( + header, + input_streams_count, + std::move(rows_sources_read_buf), + merge_block_size_rows, + merge_block_size_bytes, + is_result_sparse); + + pipeline.addTransform(std::move(transform)); + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + } + +private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + + MergeTreeData::MergingParams merging_params{}; + const String rows_sources_temporary_file_name; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + const bool is_result_sparse; +}; + +MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & column_name) const +{ + /// Read from all parts + std::vector plans; for (size_t part_num = 0; part_num < global_ctx->future_part->parts.size(); ++part_num) { - Pipe pipe = createMergeTreeSequentialSource( + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, global_ctx->future_part->parts[part_num], global_ctx->alter_conversions[part_num], Names{column_name}, - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + std::nullopt, ctx->read_with_direct_io, - ctx->use_prefetch); + ctx->use_prefetch, + global_ctx->context, + getLogger("VerticalMergeStage")); - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } - return Pipe::unitePipes(std::move(pipes)); + QueryPlan merge_column_query_plan; + + /// Union of all parts streams + { + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_column_query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + + /// Add column gatherer step + { + bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; + const auto data_settings = global_ctx->data->getSettings(); + auto merge_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + RowsSourcesTemporaryFile::FILE_ID, + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + is_result_sparse); + merge_step->setStepDescription("Gather column"); + merge_column_query_plan.addStep(std::move(merge_step)); + } + + /// Add expression step for indexes + MergeTreeIndices indexes_to_recalc; + IndicesDescription indexes_to_recalc_description; + { + auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); + + if (indexes_it != global_ctx->skip_indexes_by_column.end()) + { + indexes_to_recalc_description = indexes_it->second; + indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); + + auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. + auto calculate_indices_expression_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_column_query_plan.addStep(std::move(calculate_indices_expression_step)); + } + } + + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file; + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + + return {QueryPipelineBuilder::getPipeline(std::move(*builder)), std::move(indexes_to_recalc)}; } void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const @@ -848,50 +1010,22 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); - Pipe pipe; - if (ctx->prepared_pipe) + VerticalMergeRuntimeContext::PreparedColumnPipeline column_pipepline; + if (ctx->prepared_pipeline) { - pipe = std::move(*ctx->prepared_pipe); + column_pipepline = std::move(*ctx->prepared_pipeline); + /// Prepare next column pipeline to initiate prefetching auto next_column_it = std::next(ctx->it_name_and_type); if (next_column_it != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); + ctx->prepared_pipeline = createPipelineForReadingOneColumn(next_column_it->name); } else { - pipe = createPipeForReadingOneColumn(column_name); + column_pipepline = createPipelineForReadingOneColumn(column_name); } - ctx->rows_sources_read_buf->seek(0, 0); - bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; - - const auto data_settings = global_ctx->data->getSettings(); - auto transform = std::make_unique( - pipe.getHeader(), - pipe.numOutputPorts(), - *ctx->rows_sources_read_buf, - data_settings->merge_max_block_size, - data_settings->merge_max_block_size_bytes, - is_result_sparse); - - pipe.addTransform(std::move(transform)); - - MergeTreeIndices indexes_to_recalc; - auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); - - if (indexes_it != global_ctx->skip_indexes_by_column.end()) - { - indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); - - pipe.addTransform(std::make_shared( - pipe.getHeader(), - indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); - - pipe.addTransform(std::make_shared(pipe.getHeader())); - } - - ctx->column_parts_pipeline = QueryPipeline(std::move(pipe)); + ctx->column_parts_pipeline = std::move(column_pipepline.pipeline); /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( @@ -909,7 +1043,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->metadata_snapshot, columns_list, ctx->compression_codec, - indexes_to_recalc, + column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); @@ -1219,12 +1353,208 @@ bool MergeTask::execute() } -void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() +/// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream +class MergePartsStep : public ITransformingStep +{ +public: + MergePartsStep( + const DataStream & input_stream_, + const SortDescription & sort_description_, + const Names partition_key_columns_, + const MergeTreeData::MergingParams & merging_params_, + const String & rows_sources_temporary_file_name_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool blocks_are_granules_size_, + bool cleanup_, + time_t time_of_merge_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + , sort_description(sort_description_) + , partition_key_columns(partition_key_columns_) + , merging_params(merging_params_) + , rows_sources_temporary_file_name(rows_sources_temporary_file_name_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , blocks_are_granules_size(blocks_are_granules_size_) + , cleanup(cleanup_) + , time_of_merge(time_of_merge_) + {} + + String getName() const override { return "MergeParts"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & pipeline_settings) override + { + /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. + /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, + /// that is going in insertion order. + ProcessorPtr merged_transform; + + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + WriteBuffer * rows_sources_write_buf = nullptr; + if (!rows_sources_temporary_file_name.empty()) + { + if (!pipeline_settings.temporary_file_lookup) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file lookup is not set in pipeline settings for vertical merge"); + rows_sources_write_buf = &pipeline_settings.temporary_file_lookup->getTemporaryFileForWriting(rows_sources_temporary_file_name); + } + + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + merged_transform = std::make_shared( + header, + input_streams_count, + sort_description, + merge_block_size_rows, + merge_block_size_bytes, + SortingQueueStrategy::Default, + /* limit_= */0, + /* always_read_till_end_= */false, + rows_sources_write_buf, + blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Collapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, false, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Summing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Aggregating: + merged_transform = std::make_shared(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Replacing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.is_deleted_column, merging_params.version_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size, + cleanup); + break; + + case MergeTreeData::MergingParams::Graphite: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, + merging_params.graphite_params, time_of_merge); + break; + + case MergeTreeData::MergingParams::VersionedCollapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + } + + pipeline.addTransform(std::move(merged_transform)); + +#ifndef NDEBUG + if (!sort_description.empty()) + { + pipeline.addSimpleTransform([&](const Block & header_) + { + auto transform = std::make_shared(header_, sort_description); + return transform; + }); + } +#endif + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + } + +private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + + const SortDescription sort_description; + const Names partition_key_columns; + const MergeTreeData::MergingParams merging_params{}; + const String rows_sources_temporary_file_name; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + const bool blocks_are_granules_size; + const bool cleanup{false}; + const time_t time_of_merge{0}; +}; + +class TTLStep : public ITransformingStep +{ +public: + TTLStep( + const DataStream & input_stream_, + const ContextPtr & context_, + const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const MergeTreeData::MutableDataPartPtr & data_part_, + time_t current_time, + bool force_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + { + transform = std::make_shared(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_); + subqueries_for_sets = transform->getSubqueries(); + } + + String getName() const override { return "TTL"; } + + PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + pipeline.addTransform(transform); + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + } + +private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + + std::shared_ptr transform; + PreparedSets::Subqueries subqueries_for_sets; +}; + + +void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const { /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ - Pipes pipes; + global_ctx->watch_prev_elapsed = 0; /// We count total amount of bytes in parts @@ -1251,143 +1581,92 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); + /// Read from all parts + std::vector plans; for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { - Pipe pipe = createMergeTreeSequentialSource( + if (global_ctx->future_part->parts[i]->getMarksCount() == 0) + LOG_TRACE(ctx->log, "Part {} is empty", global_ctx->future_part->parts[i]->name); + + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, global_ctx->future_part->parts[i], global_ctx->alter_conversions[i], global_ctx->merging_columns.getNames(), - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + /*filter=*/ std::nullopt, ctx->read_with_direct_io, - /*prefetch=*/ false); + /*prefetch=*/ false, + global_ctx->context, + ctx->log); - if (global_ctx->metadata_snapshot->hasSortingKey()) - { - pipe.addSimpleTransform([this](const Block & header) - { - return std::make_shared(header, global_ctx->metadata_snapshot->getSortingKey().expression); - }); - } - - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } + QueryPlan merge_parts_query_plan; - Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; - sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; - - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; - - Block header = pipes.at(0).getHeader(); - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(sort_columns[i], 1, 1); - -#ifndef NDEBUG - if (!sort_description.empty()) + /// Union of all parts streams { - for (size_t i = 0; i < pipes.size(); ++i) - { - auto & pipe = pipes[i]; - pipe.addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - transform->setDescription(global_ctx->future_part->parts[i]->name); - return transform; - }); - } + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); } -#endif - /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. - /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, - /// that is going in insertion order. - ProcessorPtr merged_transform; - - /// If merge is vertical we cannot calculate it - ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); - - /// There is no sense to have the block size bigger than one granule for merge operations. - const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; - const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; - - switch (ctx->merging_params.mode) + if (global_ctx->metadata_snapshot->hasSortingKey()) { - case MergeTreeData::MergingParams::Ordinary: - merged_transform = std::make_shared( - header, - pipes.size(), - sort_description, - merge_block_size_rows, - merge_block_size_bytes, - SortingQueueStrategy::Default, - /* limit_= */0, - /* always_read_till_end_= */false, - ctx->rows_sources_write_buf.get(), - ctx->blocks_are_granules_size); - break; - - case MergeTreeData::MergingParams::Collapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, false, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; - - case MergeTreeData::MergingParams::Summing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); - break; - - case MergeTreeData::MergingParams::Aggregating: - merged_transform = std::make_shared(header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes); - break; - - case MergeTreeData::MergingParams::Replacing: - if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); - - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.is_deleted_column, ctx->merging_params.version_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size, - global_ctx->cleanup); - break; - - case MergeTreeData::MergingParams::Graphite: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes, - ctx->merging_params.graphite_params, global_ctx->time_of_merge); - break; - - case MergeTreeData::MergingParams::VersionedCollapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; + /// Calculate sorting key expressions so that they are available for merge sorting. + auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); + auto calculate_sorting_key_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(sorting_key_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_sorting_key_expression_step)); } - auto builder = std::make_unique(); - builder->init(Pipe::unitePipes(std::move(pipes))); - builder->addTransform(std::move(merged_transform)); - -#ifndef NDEBUG - if (!sort_description.empty()) + /// Merge { - builder->addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - return transform; - }); + Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; + sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; + + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); + + Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; + + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(sort_columns[i], 1, 1); + + const bool is_vertical_merge = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); + /// If merge is vertical we cannot calculate it + ctx->blocks_are_granules_size = is_vertical_merge; + + if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); + + auto merge_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + sort_description, + partition_key_columns, + global_ctx->merging_params, + (is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources temporaty file is used only for vertical merge + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + ctx->blocks_are_granules_size, + global_ctx->cleanup, + global_ctx->time_of_merge); + merge_step->setStepDescription("Merge sorted parts"); + merge_parts_query_plan.addStep(std::move(merge_step)); } -#endif if (global_ctx->deduplicate) { @@ -1406,37 +1685,51 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() } } - if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) - builder->addTransform(std::make_shared( - builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); - else - builder->addTransform(std::make_shared( - builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); + auto deduplication_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + SizeLimits(), 0 /*limit_hint*/, + global_ctx->deduplicate_by_columns, + false /*pre_distinct*/, + true /*optimize_distinct_in_order TODO: looks like it should be enabled*/); + deduplication_step->setStepDescription("Deduplication step"); + merge_parts_query_plan.addStep(std::move(deduplication_step)); } PreparedSets::Subqueries subqueries; + /// TTL step if (ctx->need_remove_expired_values) { - auto transform = std::make_shared(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); - subqueries = transform->getSubqueries(); - builder->addTransform(std::move(transform)); + auto ttl_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), global_ctx->context, *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); + subqueries = ttl_step->getSubqueries(); + ttl_step->setStepDescription("TTL step"); + merge_parts_query_plan.addStep(std::move(ttl_step)); } + /// Secondary indices expressions if (!global_ctx->merging_skip_indexes.empty()) { - builder->addTransform(std::make_shared( - builder->getHeader(), - global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); - - builder->addTransform(std::make_shared(builder->getHeader())); + auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. + auto calculate_indices_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step)); } if (!subqueries.empty()) - builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); + addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context); + + { + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + pipeline_settings.temporary_file_lookup = ctx->rows_sources_temporary_file; + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + + global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + } - global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); /// Is calculated inside MergeProgressCallback. @@ -1475,10 +1768,10 @@ MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm } bool is_supported_storage = - ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary || - ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || - ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || - ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; + global_ctx->merging_params.mode == MergeTreeData::MergingParams::Ordinary || + global_ctx->merging_params.mode == MergeTreeData::MergingParams::Collapsing || + global_ctx->merging_params.mode == MergeTreeData::MergingParams::Replacing || + global_ctx->merging_params.mode == MergeTreeData::MergingParams::VersionedCollapsing; bool enough_ordinary_cols = global_ctx->gathering_columns.size() >= data_settings->vertical_merge_algorithm_min_columns_to_activate; diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 9450fda7b08..29b5c4452e7 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -40,6 +40,7 @@ namespace DB class MergeTask; using MergeTaskPtr = std::shared_ptr; +class RowsSourcesTemporaryFile; /** * Overview of the merge algorithm @@ -100,6 +101,7 @@ public: global_ctx->context = std::move(context_); global_ctx->holder = &holder; global_ctx->space_reservation = std::move(space_reservation_); + global_ctx->disk = global_ctx->space_reservation->getDisk(); global_ctx->deduplicate = std::move(deduplicate_); global_ctx->deduplicate_by_columns = std::move(deduplicate_by_columns_); global_ctx->cleanup = std::move(cleanup_); @@ -110,12 +112,10 @@ public: global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_); global_ctx->txn = std::move(txn); global_ctx->need_prefix = need_prefix; + global_ctx->suffix = std::move(suffix_); + global_ctx->merging_params = std::move(merging_params_); auto prepare_stage_ctx = std::make_shared(); - - prepare_stage_ctx->suffix = std::move(suffix_); - prepare_stage_ctx->merging_params = std::move(merging_params_); - (*stages.begin())->setRuntimeContext(std::move(prepare_stage_ctx), global_ctx); } @@ -172,6 +172,7 @@ private: ContextPtr context{nullptr}; time_t time_of_merge{0}; ReservationSharedPtr space_reservation{nullptr}; + DiskPtr disk{nullptr}; bool deduplicate{false}; Names deduplicate_by_columns{}; bool cleanup{false}; @@ -210,6 +211,8 @@ private: MergeTreeTransactionPtr txn; bool need_prefix; + String suffix; + MergeTreeData::MergingParams merging_params{}; scope_guard temporary_directory_lock; UInt64 prev_elapsed_ms{0}; @@ -222,19 +225,11 @@ private: /// Proper initialization is responsibility of the author struct ExecuteAndFinalizeHorizontalPartRuntimeContext : public IStageRuntimeContext { - /// Dependencies - String suffix; - bool need_prefix; - MergeTreeData::MergingParams merging_params{}; - - TemporaryDataOnDiskPtr tmp_disk{nullptr}; - DiskPtr disk{nullptr}; bool need_remove_expired_values{false}; bool force_ttl{false}; CompressionCodecPtr compression_codec{nullptr}; size_t sum_input_rows_upper_bound{0}; - std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; - std::unique_ptr rows_sources_write_buf{nullptr}; + std::shared_ptr rows_sources_temporary_file; std::optional column_sizes{}; /// For projections to rebuild @@ -264,17 +259,16 @@ private: using ExecuteAndFinalizeHorizontalPartRuntimeContextPtr = std::shared_ptr; - struct ExecuteAndFinalizeHorizontalPart : public IStage { bool execute() override; - bool prepare(); - bool executeImpl(); + bool prepare() const; + bool executeImpl() const; void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable - using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; + using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; const ExecuteAndFinalizeHorizontalPartSubtasks subtasks { @@ -289,10 +283,10 @@ private: void calculateProjections(const Block & block) const; void finalizeProjections() const; void constructTaskForProjectionPartsMerge() const; - bool executeMergeProjections(); + bool executeMergeProjections() const; MergeAlgorithm chooseMergeAlgorithm() const; - void createMergedStream(); + void createMergedStream() const; void extractMergingAndGatheringColumns() const; void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override @@ -314,11 +308,9 @@ private: struct VerticalMergeRuntimeContext : public IStageRuntimeContext { /// Begin dependencies from previous stage - std::unique_ptr rows_sources_uncompressed_write_buf{nullptr}; - std::unique_ptr rows_sources_write_buf{nullptr}; + std::shared_ptr rows_sources_temporary_file; std::optional column_sizes; CompressionCodecPtr compression_codec; - TemporaryDataOnDiskPtr tmp_disk{nullptr}; std::list::const_iterator it_name_and_type; bool read_with_direct_io{false}; bool need_sync{false}; @@ -334,20 +326,27 @@ private: Float64 progress_before = 0; std::unique_ptr column_to{nullptr}; - std::optional prepared_pipe; + + /// Used for prefetching. Right before starting merge of a column we create a pipeline for the next column + /// and it initiates prefetching of the first range of that column. + struct PreparedColumnPipeline + { + QueryPipeline pipeline; + MergeTreeIndices indexes_to_recalc; + }; + + std::optional prepared_pipeline; size_t max_delayed_streams = 0; bool use_prefetch = false; std::list> delayed_streams; size_t column_elems_written{0}; QueryPipeline column_parts_pipeline; std::unique_ptr executor; - std::unique_ptr rows_sources_read_buf{nullptr}; UInt64 elapsed_execute_ns{0}; }; using VerticalMergeRuntimeContextPtr = std::shared_ptr; - struct VerticalMergeStage : public IStage { bool execute() override; @@ -379,7 +378,7 @@ private: bool executeVerticalMergeForOneColumn() const; void finalizeVerticalMergeForOneColumn() const; - Pipe createPipeForReadingOneColumn(const String & column_name) const; + VerticalMergeRuntimeContext::PreparedColumnPipeline createPipelineForReadingOneColumn(const String & column_name) const; VerticalMergeRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index a120716d180..78ba02aa7ac 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -354,8 +354,11 @@ public: MergeTreeData::DataPartPtr data_part_, AlterConversionsPtr alter_conversions_, Names columns_to_read_, + std::shared_ptr> filtered_rows_count_, bool apply_deleted_mask_, std::optional filter_, + bool read_with_direct_io_, + bool prefetch_, ContextPtr context_, LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) @@ -365,8 +368,11 @@ public: , data_part(std::move(data_part_)) , alter_conversions(std::move(alter_conversions_)) , columns_to_read(std::move(columns_to_read_)) + , filtered_rows_count(std::move(filtered_rows_count_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) + , read_with_direct_io(read_with_direct_io_) + , prefetch(prefetch_) , context(std::move(context_)) , log(log_) { @@ -410,25 +416,28 @@ public: alter_conversions, columns_to_read, std::move(mark_ranges), - /*filtered_rows_count=*/ nullptr, + filtered_rows_count, apply_deleted_mask, - /*read_with_direct_io=*/ false, - /*prefetch=*/ false); + read_with_direct_io, + prefetch); pipeline.init(Pipe(std::move(source))); } private: - MergeTreeSequentialSourceType type; + const MergeTreeSequentialSourceType type; const MergeTreeData & storage; - StorageSnapshotPtr storage_snapshot; - MergeTreeData::DataPartPtr data_part; - AlterConversionsPtr alter_conversions; - Names columns_to_read; - bool apply_deleted_mask; - std::optional filter; - ContextPtr context; - LoggerPtr log; + const StorageSnapshotPtr storage_snapshot; + const MergeTreeData::DataPartPtr data_part; + const AlterConversionsPtr alter_conversions; + const Names columns_to_read; + const std::shared_ptr> filtered_rows_count; + const bool apply_deleted_mask; + const std::optional filter; + const bool read_with_direct_io; + const bool prefetch; + const ContextPtr context; + const LoggerPtr log; }; void createReadFromPartStep( @@ -439,16 +448,28 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, AlterConversionsPtr alter_conversions, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log) { - auto reading = std::make_unique(type, - storage, storage_snapshot, - std::move(data_part), std::move(alter_conversions), - std::move(columns_to_read), apply_deleted_mask, - std::move(filter), std::move(context), log); + auto reading = std::make_unique( + type, + storage, + storage_snapshot, + std::move(data_part), + std::move(alter_conversions), + std::move(columns_to_read), + filtered_rows_count, + apply_deleted_mask, + std::move(filter), + read_with_direct_io, + prefetch, + std::move(context), + log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 5b7c80385f6..d2ed1394dbd 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -39,8 +39,11 @@ void createReadFromPartStep( MergeTreeData::DataPartPtr data_part, AlterConversionsPtr alter_conversions, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ee47fe3549a..8abf735b49f 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -185,7 +185,8 @@ public: void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { if (++sent_initial_requests > replicas_count) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas: replica_num={}", announcement.replica_num); doHandleInitialAllRangesAnnouncement(std::move(announcement)); } @@ -433,9 +434,9 @@ void DefaultCoordinator::setProgressCallback() void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - const auto replica_num = announcement.replica_num; + LOG_DEBUG(log, "Initial request: {}", announcement.describe()); - LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe()); + const auto replica_num = announcement.replica_num; initializeReadingState(std::move(announcement)); @@ -839,6 +840,7 @@ public: Parts all_parts_to_read; size_t total_rows_to_read = 0; + bool state_initialized{false}; LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; @@ -858,7 +860,9 @@ void InOrderCoordinator::markReplicaAsUnavailable(size_t replica_number) template void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - LOG_TRACE(log, "Received an announcement {}", announcement.describe()); + LOG_TRACE(log, "Received an announcement : {}", announcement.describe()); + + ++stats[announcement.replica_num].number_of_requests; size_t new_rows_to_read = 0; @@ -868,13 +872,16 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info == part.info; }); - /// We have the same part - add the info about presence on current replica to it + /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) { the_same_it->replicas.insert(announcement.replica_num); continue; } + if (state_initialized) + continue; + auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); @@ -889,9 +896,10 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } - ++stats[announcement.replica_num].number_of_requests; + state_initialized = true; - if (new_rows_to_read > 0) + // progress_callback is not set when local plan is used for initiator + if (progress_callback && new_rows_to_read > 0) { Progress progress; progress.total_rows_to_read = new_rows_to_read; @@ -911,7 +919,7 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest "Replica {} decided to read in {} mode, not in {}. This is a bug", request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode)); - LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe()); + LOG_TRACE(log, "Got read request: {}", request.describe()); ParallelReadResponse response; response.description = request.description; @@ -925,8 +933,15 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest if (global_part_it == all_parts_to_read.end()) continue; + if (global_part_it->replicas.empty()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Part {} requested by replica {} is not registered in working set", + part.info.getPartNameV1(), + request.replica_num); + if (!global_part_it->replicas.contains(request.replica_num)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num); + continue; size_t current_mark_size = 0; @@ -1057,6 +1072,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) break; } + // progress_callback is not set when local plan is used for initiator if (progress_callback) pimpl->setProgressCallback(std::move(progress_callback)); diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 2ce0e20dcd2..bcdeb443a0b 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -44,9 +44,7 @@ void ParallelReadRequest::serialize(WriteBuffer & out) const String ParallelReadRequest::describe() const { - String result; - result += fmt::format("replica_num: {} \n", replica_num); - result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks); + String result = fmt::format("replica_num {}, min_num_of_marks {}, ", replica_num, min_number_of_marks); result += description.describe(); return result; } @@ -131,10 +129,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const String InitialAllRangesAnnouncement::describe() { - String result; - result += description.describe(); - result += fmt::format("----------\nReceived from {} replica\n", replica_num); - return result; + return fmt::format("replica {}, mode {}, {}", replica_num, mode, description.describe()); } InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index 9730391d429..8121f389a8d 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } const std::unordered_set required_configuration_keys = { @@ -146,14 +147,13 @@ void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & coll void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) { - if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) + if (engine_args.size() < 3 || engine_args.size() > getMaxNumberOfArguments(with_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)); + "Storage AzureBlobStorage requires 1 to {} arguments. All supported signatures:\n{}", + getMaxNumberOfArguments(with_structure), + getSignatures(with_structure)); } for (auto & engine_arg : engine_args) @@ -272,26 +272,30 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, connection_params = getConnectionParams(connection_url, container_name, account_name, account_key, context); } -void StorageAzureConfiguration::addStructureAndFormatToArgs( +void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, ContextPtr context) { - if (tryGetNamedCollectionWithOverrides(args, context)) + if (auto collection = tryGetNamedCollectionWithOverrides(args, context)) { - /// In case of named collection, just add key-value pair "structure='...'" - /// at the end of arguments to override existed structure. - ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; - auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); - args.push_back(equal_func); + /// In case of named collection, just add key-value pairs "format='...', structure='...'" + /// at the end of arguments to override existed format and structure with "auto" values. + if (collection->getOrDefault("format", "auto") == "auto") + { + ASTs format_equal_func_args = {std::make_shared("format"), std::make_shared(format_)}; + auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args)); + args.push_back(format_equal_func); + } + if (collection->getOrDefault("structure", "auto") == "auto") + { + ASTs structure_equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args)); + args.push_back(structure_equal_func); + } } else { - if (args.size() < 3 || args.size() > 8) - { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "StorageObjectStorage(connection_string|storage_account_url, container_name, " - "blobpath, [account_name, account_key, format, compression, structure])"); - } + if (args.size() < 3 || args.size() > getMaxNumberOfArguments()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 3 to {} arguments in table function azureBlobStorage, got {}", getMaxNumberOfArguments(), args.size()); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 4e6bfbc0745..c3adc86b124 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -22,6 +22,29 @@ public: static constexpr auto type_name = "azure"; static constexpr auto engine_name = "Azure"; + /// All possible signatures for Azure engine with structure argument (for example for azureBlobStorage table function). + static constexpr auto max_number_of_arguments_with_structure = 8; + static constexpr auto signatures_with_structure = + " - connection_string, container_name, blobpath\n" + " - connection_string, container_name, blobpath, structure \n" + " - connection_string, container_name, blobpath, format \n" + " - connection_string, container_name, blobpath, format, compression \n" + " - connection_string, container_name, blobpath, format, compression, structure \n" + " - storage_account_url, container_name, blobpath, account_name, account_key\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; + + /// All possible signatures for Azure engine without structure argument (for example for AzureBlobStorage table engine). + static constexpr auto max_number_of_arguments_without_structure = 7; + static constexpr auto signatures_without_structure = + " - connection_string, container_name, blobpath\n" + " - connection_string, container_name, blobpath, format \n" + " - connection_string, container_name, blobpath, format, compression \n" + " - storage_account_url, container_name, blobpath, account_name, account_key\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n"; StorageAzureConfiguration() = default; StorageAzureConfiguration(const StorageAzureConfiguration & other); @@ -29,6 +52,9 @@ public: std::string getTypeName() const override { return type_name; } std::string getEngineName() const override { return engine_name; } + std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; } + size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; } + Path getPath() const override { return blob_path; } void setPath(const Path & path) override { blob_path = path; } @@ -44,7 +70,7 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; - void addStructureAndFormatToArgs( + void addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 85eb29a3868..9b5bbdeacc1 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) @@ -83,12 +84,13 @@ StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(c void StorageHDFSConfiguration::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); - } + if (args.empty() || args.size() > getMaxNumberOfArguments(with_structure)) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage HDFS requires 1 to {} arguments. All supported signatures:\n{}", + getMaxNumberOfArguments(with_structure), + getSignatures(with_structure)); + std::string url_str; url_str = checkAndGetLiteralArgument(args[0], "url"); @@ -158,28 +160,34 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using URL: {}, path: {}", url, path); } -void StorageHDFSConfiguration::addStructureAndFormatToArgs( +void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, ContextPtr context) { - if (tryGetNamedCollectionWithOverrides(args, context)) + if (auto collection = tryGetNamedCollectionWithOverrides(args, context)) { - /// In case of named collection, just add key-value pair "structure='...'" - /// at the end of arguments to override existed structure. - ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; - auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); - args.push_back(equal_func); + /// In case of named collection, just add key-value pairs "format='...', structure='...'" + /// at the end of arguments to override existed format and structure with "auto" values. + if (collection->getOrDefault("format", "auto") == "auto") + { + ASTs format_equal_func_args = {std::make_shared("format"), std::make_shared(format_)}; + auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args)); + args.push_back(format_equal_func); + } + if (collection->getOrDefault("structure", "auto") == "auto") + { + ASTs structure_equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args)); + args.push_back(structure_equal_func); + } } else { size_t count = args.size(); - if (count == 0 || count > 4) - { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Expected 1 to 4 arguments in table function, got {}", count); - } + if (count == 0 || count > getMaxNumberOfArguments()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function hdfs, got {}", getMaxNumberOfArguments(), count); auto format_literal = std::make_shared(format_); auto structure_literal = std::make_shared(structure_); diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 04884542908..206147d7e5e 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -16,6 +16,20 @@ public: static constexpr auto type_name = "hdfs"; static constexpr auto engine_name = "HDFS"; + /// All possible signatures for HDFS engine with structure argument (for example for hdfs table function). + static constexpr auto max_number_of_arguments_with_structure = 4; + static constexpr auto signatures_with_structure = + " - uri\n" + " - uri, format\n" + " - uri, format, structure\n" + " - uri, format, structure, compression_method\n"; + + /// All possible signatures for HDFS engine without structure argument (for example for HS table engine). + static constexpr auto max_number_of_arguments_without_structure = 3; + static constexpr auto signatures_without_structure = + " - uri\n" + " - uri, format\n" + " - uri, format, compression_method\n"; StorageHDFSConfiguration() = default; StorageHDFSConfiguration(const StorageHDFSConfiguration & other); @@ -23,6 +37,9 @@ public: std::string getTypeName() const override { return type_name; } std::string getEngineName() const override { return engine_name; } + std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; } + size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; } + Path getPath() const override { return path; } void setPath(const Path & path_) override { path = path_; } @@ -39,7 +56,7 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; - void addStructureAndFormatToArgs( + void addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, diff --git a/src/Storages/ObjectStorage/Local/Configuration.cpp b/src/Storages/ObjectStorage/Local/Configuration.cpp index a0cf70e6212..0554b9c317c 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.cpp +++ b/src/Storages/ObjectStorage/Local/Configuration.cpp @@ -26,11 +26,11 @@ void StorageLocalConfiguration::fromNamedCollection(const NamedCollection & coll 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); - } + if (args.empty() || args.size() > getMaxNumberOfArguments(with_structure)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Local requires 1 to {} arguments. All supported signatures:\n{}", + getMaxNumberOfArguments(with_structure), + getSignatures(with_structure)); for (auto & arg : args) arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); diff --git a/src/Storages/ObjectStorage/Local/Configuration.h b/src/Storages/ObjectStorage/Local/Configuration.h index ba4de63ac47..84dc3855df3 100644 --- a/src/Storages/ObjectStorage/Local/Configuration.h +++ b/src/Storages/ObjectStorage/Local/Configuration.h @@ -19,6 +19,20 @@ public: using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; static constexpr auto type_name = "local"; + /// All possible signatures for Local engine with structure argument (for example for local table function). + static constexpr auto max_number_of_arguments_with_structure = 4; + static constexpr auto signatures_with_structure = + " - path\n" + " - path, format\n" + " - path, format, structure\n" + " - path, format, structure, compression_method\n"; + + /// All possible signatures for S3 engine without structure argument (for example for Local table engine). + static constexpr auto max_number_of_arguments_without_structure = 3; + static constexpr auto signatures_without_structure = + " - path\n" + " - path, format\n" + " - path, format, compression_method\n"; StorageLocalConfiguration() = default; StorageLocalConfiguration(const StorageLocalConfiguration & other) = default; @@ -26,6 +40,9 @@ public: std::string getTypeName() const override { return type_name; } std::string getEngineName() const override { return "Local"; } + std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; } + size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; } + Path getPath() const override { return path; } void setPath(const Path & path_) override { path = path_; } @@ -40,7 +57,7 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared("/"); } - void addStructureAndFormatToArgs(ASTs &, const String &, const String &, ContextPtr) override { } + void addStructureAndFormatToArgsIfNeeded(ASTs &, const String &, const String &, ContextPtr) override { } private: void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 7542f59dcc4..56bc6ea2f61 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -170,21 +170,20 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) { - /// Supported signatures: S3('url') S3('url', 'format') S3('url', 'format', 'compression') S3('url', NOSIGN) S3('url', NOSIGN, 'format') S3('url', NOSIGN, 'format', 'compression') S3('url', 'aws_access_key_id', 'aws_secret_access_key') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression') - /// with optional headers() function - size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context); - if (count == 0 || count > (with_structure ? 7 : 6)) + if (count == 0 || count > getMaxNumberOfArguments(with_structure)) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage S3 requires 1 to 5 arguments: " - "url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]"); + "Storage S3 requires 1 to {} arguments. All supported signatures:\n{}", + getMaxNumberOfArguments(with_structure), + getSignatures(with_structure)); std::unordered_map engine_args_to_idx; bool no_sign_request = false; - /// For 2 arguments we support 2 possible variants: + /// When adding new arguments in the signature don't forget to update addStructureAndFormatToArgsIfNeeded as well. + + /// For 2 arguments we support: /// - s3(source, format) /// - s3(source, NOSIGN) /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. @@ -196,10 +195,15 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ else engine_args_to_idx = {{"format", 1}}; } - /// For 3 arguments we support 2 possible variants: + /// For 3 arguments we support: + /// if with_structure == 0: + /// - s3(source, NOSIGN, format) /// - s3(source, format, compression_method) /// - s3(source, access_key_id, secret_access_key) + /// if with_structure == 1: /// - s3(source, NOSIGN, format) + /// - s3(source, format, structure) + /// - s3(source, access_key_id, secret_access_key) /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. else if (count == 3) { @@ -219,7 +223,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ else engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; } - /// For 4 arguments we support 3 possible variants: + /// For 4 arguments we support: /// if with_structure == 0: /// - s3(source, access_key_id, secret_access_key, session_token) /// - s3(source, access_key_id, secret_access_key, format) @@ -229,7 +233,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ /// - s3(source, access_key_id, secret_access_key, format), /// - s3(source, access_key_id, secret_access_key, session_token) /// - s3(source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN, format name of something else. else if (count == 4) { auto second_arg = checkAndGetLiteralArgument(args[1], "access_key_id/NOSIGN"); @@ -258,7 +262,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ } } } - /// For 5 arguments we support 2 possible variants: + /// For 5 arguments we support: /// if with_structure == 0: /// - s3(source, access_key_id, secret_access_key, session_token, format) /// - s3(source, access_key_id, secret_access_key, format, compression) @@ -302,13 +306,16 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ } } } + /// For 6 arguments we support: + /// if with_structure == 0: + /// - s3(source, access_key_id, secret_access_key, session_token, format, compression_method) + /// if with_structure == 1: + /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) + /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) else if (count == 6) { if (with_structure) { - /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) - /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) - /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) { @@ -324,6 +331,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}; } } + /// s3(source, access_key_id, secret_access_key, session_token, format, structure, compression_method) else if (with_structure && count == 7) { engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}; @@ -365,24 +373,33 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ keys = {url.key}; } -void StorageS3Configuration::addStructureAndFormatToArgs( +void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, ContextPtr context) { - if (tryGetNamedCollectionWithOverrides(args, context)) + if (auto collection = tryGetNamedCollectionWithOverrides(args, context)) { - /// In case of named collection, just add key-value pair "structure='...'" - /// at the end of arguments to override existed structure. - ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; - auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); - args.push_back(equal_func); + /// In case of named collection, just add key-value pairs "format='...', structure='...'" + /// at the end of arguments to override existed format and structure with "auto" values. + if (collection->getOrDefault("format", "auto") == "auto") + { + ASTs format_equal_func_args = {std::make_shared("format"), std::make_shared(format_)}; + auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args)); + args.push_back(format_equal_func); + } + if (collection->getOrDefault("structure", "auto") == "auto") + { + ASTs structure_equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args)); + args.push_back(structure_equal_func); + } } else { HTTPHeaderEntries tmp_headers; size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); - if (count == 0 || count > 6) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to 6 arguments in table function, got {}", count); + if (count == 0 || count > getMaxNumberOfArguments()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function s3, got {}", getMaxNumberOfArguments(), count); auto format_literal = std::make_shared(format_); auto structure_literal = std::make_shared(structure_); @@ -394,14 +411,18 @@ void StorageS3Configuration::addStructureAndFormatToArgs( args.push_back(std::make_shared("auto")); args.push_back(structure_literal); } - /// s3(s3_url, format) or s3(s3_url, NOSIGN) + /// s3(s3_url, format) or + /// s3(s3_url, NOSIGN) /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. else if (count == 2) { auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); /// If there is NOSIGN, add format=auto before structure. if (boost::iequals(second_arg, "NOSIGN")) - args.push_back(std::make_shared("auto")); + args.push_back(format_literal); + else if (checkAndGetLiteralArgument(args[1], "format") == "auto") + args[1] = format_literal; + args.push_back(structure_literal); } /// s3(source, format, structure) or @@ -413,21 +434,27 @@ void StorageS3Configuration::addStructureAndFormatToArgs( auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); if (boost::iequals(second_arg, "NOSIGN")) { + if (checkAndGetLiteralArgument(args[2], "format") == "auto") + args[2] = format_literal; args.push_back(structure_literal); } else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) { - args[count - 1] = structure_literal; + if (second_arg == "auto") + args[1] = format_literal; + if (checkAndGetLiteralArgument(args[2], "structure") == "auto") + args[2] = structure_literal; } else { - /// Add format=auto before structure argument. - args.push_back(std::make_shared("auto")); + /// Add format and structure arguments. + args.push_back(format_literal); args.push_back(structure_literal); } } /// s3(source, format, structure, compression_method) or /// s3(source, access_key_id, secret_access_key, format) or + /// s3(source, access_key_id, secret_access_key, session_token) or /// s3(source, NOSIGN, format, structure) /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. else if (count == 4) @@ -435,36 +462,93 @@ void StorageS3Configuration::addStructureAndFormatToArgs( auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); if (boost::iequals(second_arg, "NOSIGN")) { - args[count - 1] = structure_literal; + if (checkAndGetLiteralArgument(args[2], "format") == "auto") + args[2] = format_literal; + if (checkAndGetLiteralArgument(args[3], "structure") == "auto") + args[3] = structure_literal; } else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg)) { - args[count - 2] = structure_literal; + if (second_arg == "auto") + args[1] = format_literal; + if (checkAndGetLiteralArgument(args[2], "structure") == "auto") + args[2] = structure_literal; } else { - args.push_back(structure_literal); + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + if (checkAndGetLiteralArgument(args[3], "format") == "auto") + args[3] = format_literal; + args.push_back(structure_literal); + } + else + { + args.push_back(format_literal); + args.push_back(structure_literal); + } } } /// s3(source, access_key_id, secret_access_key, format, structure) or + /// s3(source, access_key_id, secret_access_key, session_token, format) or /// s3(source, NOSIGN, format, structure, compression_method) /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. else if (count == 5) { - auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(sedond_arg, "NOSIGN")) + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) { - args[count - 2] = structure_literal; + if (checkAndGetLiteralArgument(args[2], "format") == "auto") + args[2] = format_literal; + if (checkAndGetLiteralArgument(args[2], "structure") == "auto") + args[3] = structure_literal; } else { - args[count - 1] = structure_literal; + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + if (checkAndGetLiteralArgument(args[3], "format") == "auto") + args[3] = format_literal; + if (checkAndGetLiteralArgument(args[4], "structure") == "auto") + args[4] = structure_literal; + } + else + { + if (checkAndGetLiteralArgument(args[4], "format") == "auto") + args[4] = format_literal; + args.push_back(structure_literal); + } } } - /// s3(source, access_key_id, secret_access_key, format, structure, compression) + /// s3(source, access_key_id, secret_access_key, format, structure, compression) or + /// s3(source, access_key_id, secret_access_key, session_token, format, structure) else if (count == 6) { - args[count - 2] = structure_literal; + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().exists(fourth_arg)) + { + if (checkAndGetLiteralArgument(args[3], "format") == "auto") + args[3] = format_literal; + if (checkAndGetLiteralArgument(args[4], "structure") == "auto") + args[4] = structure_literal; + } + else + { + if (checkAndGetLiteralArgument(args[4], "format") == "auto") + args[4] = format_literal; + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = structure_literal; + } + } + /// s3(source, access_key_id, secret_access_key, session_token, format, structure, compression_method) + else if (count == 7) + { + if (checkAndGetLiteralArgument(args[4], "format") == "auto") + args[4] = format_literal; + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = structure_literal; } } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 39a646c7df2..b36df67fb0f 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -16,6 +16,43 @@ public: static constexpr auto type_name = "s3"; static constexpr auto namespace_name = "bucket"; + /// All possible signatures for S3 storage with structure argument (for example for s3 table function). + static constexpr auto max_number_of_arguments_with_structure = 7; + static constexpr auto signatures_with_structure = + " - url\n" + " - url, NOSIGN\n" + " - url, format\n" + " - url, NOSIGN, format\n" + " - url, format, structure\n" + " - url, NOSIGN, format, structure\n" + " - url, format, structure, compression_method\n" + " - url, NOSIGN, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key\n" + " - url, access_key_id, secret_access_key, session_token\n" + " - url, access_key_id, secret_access_key, format\n" + " - url, access_key_id, secret_access_key, session_token, format\n" + " - url, access_key_id, secret_access_key, format, structure\n" + " - url, access_key_id, secret_access_key, session_token, format, structure\n" + " - url, access_key_id, secret_access_key, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + + /// All possible signatures for S3 storage without structure argument (for example for S3 table engine). + static constexpr auto max_number_of_arguments_without_structure = 6; + static constexpr auto signatures_without_structure = + " - url\n" + " - url, NOSIGN\n" + " - url, format\n" + " - url, NOSIGN, format\n" + " - url, format, compression_method\n" + " - url, NOSIGN, format, compression_method\n" + " - url, access_key_id, secret_access_key\n" + " - url, access_key_id, secret_access_key, session_token\n" + " - url, access_key_id, secret_access_key, format\n" + " - url, access_key_id, secret_access_key, session_token, format\n" + " - url, access_key_id, secret_access_key, format, compression_method\n" + " - url, access_key_id, secret_access_key, session_token, format, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; StorageS3Configuration() = default; StorageS3Configuration(const StorageS3Configuration & other); @@ -24,6 +61,9 @@ public: std::string getEngineName() const override { return url.storage_name; } std::string getNamespaceType() const override { return namespace_name; } + std::string getSignatures(bool with_structure = true) const { return with_structure ? signatures_with_structure : signatures_without_structure; } + size_t getMaxNumberOfArguments(bool with_structure = true) const { return with_structure ? max_number_of_arguments_with_structure : max_number_of_arguments_without_structure; } + Path getPath() const override { return url.key; } void setPath(const Path & path) override { url.key = path; } @@ -44,7 +84,7 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; - void addStructureAndFormatToArgs( + void addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure, const String & format, diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 562ca259089..f39586c23b4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -180,7 +180,9 @@ public: virtual String getNamespace() const = 0; virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; - virtual void addStructureAndFormatToArgs( + + /// Add/replace structure and format arguments in the AST arguments if they have 'auto' values. + virtual void addStructureAndFormatToArgsIfNeeded( ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; bool withPartitionWildcard() const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 08a0739d929..d712e4eec20 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -103,7 +103,7 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( ASTPtr cluster_name_arg = args.front(); args.erase(args.begin()); - configuration->addStructureAndFormatToArgs(args, structure, configuration->format, context); + configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context); args.insert(args.begin(), cluster_name_arg); } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp index 23ac92b667a..2da4aa6b665 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.cpp @@ -33,7 +33,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int BAD_ARGUMENTS; extern const int REPLICA_ALREADY_EXISTS; - extern const int INCOMPATIBLE_COLUMNS; } namespace @@ -108,8 +107,12 @@ private: } }; -ObjectStorageQueueMetadata::ObjectStorageQueueMetadata(const fs::path & zookeeper_path_, const ObjectStorageQueueSettings & settings_) +ObjectStorageQueueMetadata::ObjectStorageQueueMetadata( + const fs::path & zookeeper_path_, + const ObjectStorageQueueTableMetadata & table_metadata_, + const ObjectStorageQueueSettings & settings_) : settings(settings_) + , table_metadata(table_metadata_) , zookeeper_path(zookeeper_path_) , buckets_num(getBucketsNum(settings_)) , log(getLogger("StorageObjectStorageQueue(" + zookeeper_path_.string() + ")")) @@ -144,11 +147,6 @@ void ObjectStorageQueueMetadata::shutdown() task->deactivate(); } -void ObjectStorageQueueMetadata::checkSettings(const ObjectStorageQueueSettings & settings_) const -{ - ObjectStorageQueueTableMetadata::checkEquals(settings, settings_); -} - ObjectStorageQueueMetadata::FileStatusPtr ObjectStorageQueueMetadata::getFileStatus(const std::string & path) { return local_file_statuses->get(path, /* create */false); @@ -219,13 +217,14 @@ ObjectStorageQueueMetadata::tryAcquireBucket(const Bucket & bucket, const Proces return ObjectStorageQueueOrderedFileMetadata::tryAcquireBucket(zookeeper_path, bucket, processor, log); } -void ObjectStorageQueueMetadata::initialize( - const ConfigurationPtr & configuration, - const StorageInMemoryMetadata & storage_metadata) +void ObjectStorageQueueMetadata::syncWithKeeper( + const fs::path & zookeeper_path, + const ObjectStorageQueueTableMetadata & table_metadata, + const ObjectStorageQueueSettings & settings, + LoggerPtr log) { - const auto metadata_from_table = ObjectStorageQueueTableMetadata(*configuration, settings, storage_metadata); - const auto & columns_from_table = storage_metadata.getColumns(); const auto table_metadata_path = zookeeper_path / "metadata"; + const auto buckets_num = getBucketsNum(settings); const auto metadata_paths = settings.mode == ObjectStorageQueueMode::ORDERED ? ObjectStorageQueueOrderedFileMetadata::getMetadataPaths(buckets_num) : ObjectStorageQueueUnorderedFileMetadata::getMetadataPaths(); @@ -237,24 +236,19 @@ void ObjectStorageQueueMetadata::initialize( { if (zookeeper->exists(table_metadata_path)) { - const auto metadata_from_zk = ObjectStorageQueueTableMetadata::parse(zookeeper->get(fs::path(zookeeper_path) / "metadata")); - const auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); + const auto metadata_str = zookeeper->get(fs::path(zookeeper_path) / "metadata"); + const auto metadata_from_zk = ObjectStorageQueueTableMetadata::parse(metadata_str); - metadata_from_table.checkEquals(metadata_from_zk); - if (columns_from_zk != columns_from_table) - { - throw Exception( - ErrorCodes::INCOMPATIBLE_COLUMNS, - "Table columns structure in ZooKeeper is different from local table structure. " - "Local columns:\n{}\nZookeeper columns:\n{}", - columns_from_table.toString(), columns_from_zk.toString()); - } + LOG_TRACE(log, "Metadata in keeper: {}", metadata_str); + + table_metadata.checkEquals(metadata_from_zk); return; } Coordination::Requests requests; requests.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", zkutil::CreateMode::Persistent)); - requests.emplace_back(zkutil::makeCreateRequest(table_metadata_path, metadata_from_table.toString(), zkutil::CreateMode::Persistent)); + requests.emplace_back(zkutil::makeCreateRequest( + table_metadata_path, table_metadata.toString(), zkutil::CreateMode::Persistent)); for (const auto & path : metadata_paths) { @@ -263,16 +257,27 @@ void ObjectStorageQueueMetadata::initialize( } if (!settings.last_processed_path.value.empty()) - getFileMetadata(settings.last_processed_path)->setProcessedAtStartRequests(requests, zookeeper); + { + ObjectStorageQueueOrderedFileMetadata( + zookeeper_path, + settings.last_processed_path, + std::make_shared(), + /* bucket_info */nullptr, + buckets_num, + settings.loading_retries, + log).setProcessedAtStartRequests(requests, zookeeper); + } Coordination::Responses responses; auto code = zookeeper->tryMulti(requests, responses); if (code == Coordination::Error::ZNODEEXISTS) { auto exception = zkutil::KeeperMultiException(code, requests, responses); + LOG_INFO(log, "Got code `{}` for path: {}. " "It looks like the table {} was created by another server at the same moment, " - "will retry", code, exception.getPathForFirstFailedOp(), zookeeper_path.string()); + "will retry", + code, exception.getPathForFirstFailedOp(), zookeeper_path.string()); continue; } else if (code != Coordination::Error::ZOK) diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h index e5fae047ac5..71d26ca7c47 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadata.h @@ -8,6 +8,7 @@ #include #include #include +#include #include namespace fs = std::filesystem; @@ -52,11 +53,19 @@ public: using Bucket = size_t; using Processor = std::string; - ObjectStorageQueueMetadata(const fs::path & zookeeper_path_, const ObjectStorageQueueSettings & settings_); + ObjectStorageQueueMetadata( + const fs::path & zookeeper_path_, + const ObjectStorageQueueTableMetadata & table_metadata_, + const ObjectStorageQueueSettings & settings_); + ~ObjectStorageQueueMetadata(); - void initialize(const ConfigurationPtr & configuration, const StorageInMemoryMetadata & storage_metadata); - void checkSettings(const ObjectStorageQueueSettings & settings) const; + static void syncWithKeeper( + const fs::path & zookeeper_path, + const ObjectStorageQueueTableMetadata & table_metadata, + const ObjectStorageQueueSettings & settings, + LoggerPtr log); + void shutdown(); FileMetadataPtr getFileMetadata(const std::string & path, ObjectStorageQueueOrderedFileMetadata::BucketInfoPtr bucket_info = {}); @@ -72,11 +81,17 @@ public: static size_t getBucketsNum(const ObjectStorageQueueSettings & settings); static size_t getBucketsNum(const ObjectStorageQueueTableMetadata & settings); + void checkTableMetadataEquals(const ObjectStorageQueueMetadata & other); + + const ObjectStorageQueueTableMetadata & getTableMetadata() const { return table_metadata; } + ObjectStorageQueueTableMetadata & getTableMetadata() { return table_metadata; } + private: void cleanupThreadFunc(); void cleanupThreadFuncImpl(); - const ObjectStorageQueueSettings settings; + ObjectStorageQueueSettings settings; + ObjectStorageQueueTableMetadata table_metadata; const fs::path zookeeper_path; const size_t buckets_num; @@ -89,4 +104,6 @@ private: std::shared_ptr local_file_statuses; }; +using ObjectStorageQueueMetadataPtr = std::unique_ptr; + } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.cpp index ffae33d6f41..ba98711eff9 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.cpp @@ -14,19 +14,23 @@ ObjectStorageQueueMetadataFactory & ObjectStorageQueueMetadataFactory::instance( return ret; } -ObjectStorageQueueMetadataFactory::FilesMetadataPtr -ObjectStorageQueueMetadataFactory::getOrCreate(const std::string & zookeeper_path, const ObjectStorageQueueSettings & settings) +ObjectStorageQueueMetadataFactory::FilesMetadataPtr ObjectStorageQueueMetadataFactory::getOrCreate( + const std::string & zookeeper_path, + ObjectStorageQueueMetadataPtr metadata) { std::lock_guard lock(mutex); auto it = metadata_by_path.find(zookeeper_path); if (it == metadata_by_path.end()) { - auto files_metadata = std::make_shared(zookeeper_path, settings); - it = metadata_by_path.emplace(zookeeper_path, std::move(files_metadata)).first; + it = metadata_by_path.emplace(zookeeper_path, std::move(metadata)).first; } else { - it->second.metadata->checkSettings(settings); + auto & metadata_from_table = metadata->getTableMetadata(); + auto & metadata_from_keeper = it->second.metadata->getTableMetadata(); + + metadata_from_table.checkEquals(metadata_from_keeper); + it->second.ref_count += 1; } return it->second.metadata; diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h index a93f5ee3d83..a9975c526ef 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueMetadataFactory.h @@ -13,7 +13,9 @@ public: static ObjectStorageQueueMetadataFactory & instance(); - FilesMetadataPtr getOrCreate(const std::string & zookeeper_path, const ObjectStorageQueueSettings & settings); + FilesMetadataPtr getOrCreate( + const std::string & zookeeper_path, + ObjectStorageQueueMetadataPtr metadata); void remove(const std::string & zookeeper_path); diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp index cb9cdf8e186..926d5aacda4 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -32,18 +31,18 @@ namespace ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata( - const StorageObjectStorage::Configuration & configuration, const ObjectStorageQueueSettings & engine_settings, - const StorageInMemoryMetadata & storage_metadata) + const ColumnsDescription & columns_, + const std::string & format_) + : format_name(format_) + , columns(columns_.toString()) + , after_processing(engine_settings.after_processing.toString()) + , mode(engine_settings.mode.toString()) + , tracked_files_limit(engine_settings.tracked_files_limit) + , tracked_file_ttl_sec(engine_settings.tracked_file_ttl_sec) + , buckets(engine_settings.buckets) + , processing_threads_num(engine_settings.processing_threads_num) { - format_name = configuration.format; - after_processing = engine_settings.after_processing.toString(); - mode = engine_settings.mode.toString(); - tracked_files_limit = engine_settings.tracked_files_limit; - tracked_file_ttl_sec = engine_settings.tracked_file_ttl_sec; - buckets = engine_settings.buckets; - processing_threads_num = engine_settings.processing_threads_num; - columns = storage_metadata.getColumns().toString(); } String ObjectStorageQueueTableMetadata::toString() const @@ -65,48 +64,40 @@ String ObjectStorageQueueTableMetadata::toString() const return oss.str(); } -void ObjectStorageQueueTableMetadata::read(const String & metadata_str) +template +static auto getOrDefault( + const Poco::JSON::Object::Ptr & json, + const std::string & setting, + const std::string & compatibility_prefix, + const T & default_value) { - Poco::JSON::Parser parser; - auto json = parser.parse(metadata_str).extract(); + if (!compatibility_prefix.empty() && json->has(compatibility_prefix + setting)) + return json->getValue(compatibility_prefix + setting); - after_processing = json->getValue("after_processing"); - mode = json->getValue("mode"); + if (json->has(setting)) + return json->getValue(setting); - format_name = json->getValue("format_name"); - columns = json->getValue("columns"); + return default_value; +} - /// Check with "s3queue_" prefix for compatibility. - { - if (json->has("s3queue_tracked_files_limit")) - tracked_files_limit = json->getValue("s3queue_tracked_files_limit"); - if (json->has("s3queue_tracked_file_ttl_sec")) - tracked_file_ttl_sec = json->getValue("s3queue_tracked_file_ttl_sec"); - if (json->has("s3queue_processing_threads_num")) - processing_threads_num = json->getValue("s3queue_processing_threads_num"); - } - - if (json->has("tracked_files_limit")) - tracked_files_limit = json->getValue("tracked_files_limit"); - - if (json->has("tracked_file_ttl_sec")) - tracked_file_ttl_sec = json->getValue("tracked_file_ttl_sec"); - - if (json->has("last_processed_file")) - last_processed_path = json->getValue("last_processed_file"); - - if (json->has("processing_threads_num")) - processing_threads_num = json->getValue("processing_threads_num"); - - if (json->has("buckets")) - buckets = json->getValue("buckets"); +ObjectStorageQueueTableMetadata::ObjectStorageQueueTableMetadata(const Poco::JSON::Object::Ptr & json) + : format_name(json->getValue("format_name")) + , columns(json->getValue("columns")) + , after_processing(json->getValue("after_processing")) + , mode(json->getValue("mode")) + , tracked_files_limit(getOrDefault(json, "tracked_files_limit", "s3queue_", 0)) + , tracked_file_ttl_sec(getOrDefault(json, "tracked_files_ttl_sec", "s3queue_", 0)) + , buckets(getOrDefault(json, "buckets", "", 0)) + , processing_threads_num(getOrDefault(json, "processing_threads_num", "s3queue_", 1)) + , last_processed_path(getOrDefault(json, "last_processed_file", "s3queue_", "")) +{ } ObjectStorageQueueTableMetadata ObjectStorageQueueTableMetadata::parse(const String & metadata_str) { - ObjectStorageQueueTableMetadata metadata; - metadata.read(metadata_str); - return metadata; + Poco::JSON::Parser parser; + auto json = parser.parse(metadata_str).extract(); + return ObjectStorageQueueTableMetadata(json); } void ObjectStorageQueueTableMetadata::checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const @@ -181,72 +172,17 @@ void ObjectStorageQueueTableMetadata::checkImmutableFieldsEquals(const ObjectSto ErrorCodes::METADATA_MISMATCH, "Existing table metadata in ZooKeeper differs in processing buckets. " "Stored in ZooKeeper: {}, local: {}", - ObjectStorageQueueMetadata::getBucketsNum(*this), ObjectStorageQueueMetadata::getBucketsNum(from_zk)); + ObjectStorageQueueMetadata::getBucketsNum(from_zk), ObjectStorageQueueMetadata::getBucketsNum(*this)); } } + + if (columns != from_zk.columns) + throw Exception( + ErrorCodes::METADATA_MISMATCH, + "Existing table metadata in ZooKeeper differs in columns. " + "Stored in ZooKeeper: {}, local: {}", + from_zk.columns, + columns); } -void ObjectStorageQueueTableMetadata::checkEquals(const ObjectStorageQueueSettings & current, const ObjectStorageQueueSettings & expected) -{ - if (current.after_processing != expected.after_processing) - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs " - "in action after processing. Stored in ZooKeeper: {}, local: {}", - expected.after_processing.toString(), - current.after_processing.toString()); - - if (current.mode != expected.mode) - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in engine mode. " - "Stored in ZooKeeper: {}, local: {}", - expected.mode.toString(), - current.mode.toString()); - - if (current.tracked_files_limit != expected.tracked_files_limit) - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in max set size. " - "Stored in ZooKeeper: {}, local: {}", - expected.tracked_files_limit, - current.tracked_files_limit); - - if (current.tracked_file_ttl_sec != expected.tracked_file_ttl_sec) - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in max set age. " - "Stored in ZooKeeper: {}, local: {}", - expected.tracked_file_ttl_sec, - current.tracked_file_ttl_sec); - - if (current.last_processed_path.value != expected.last_processed_path.value) - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in last_processed_path. " - "Stored in ZooKeeper: {}, local: {}", - expected.last_processed_path.value, - current.last_processed_path.value); - - if (current.mode == ObjectStorageQueueMode::ORDERED) - { - if (current.buckets != expected.buckets) - { - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in buckets setting. " - "Stored in ZooKeeper: {}, local: {}", - expected.buckets, current.buckets); - } - - if (ObjectStorageQueueMetadata::getBucketsNum(current) != ObjectStorageQueueMetadata::getBucketsNum(expected)) - { - throw Exception( - ErrorCodes::METADATA_MISMATCH, - "Existing table metadata in ZooKeeper differs in processing buckets. " - "Stored in ZooKeeper: {}, local: {}", - ObjectStorageQueueMetadata::getBucketsNum(current), ObjectStorageQueueMetadata::getBucketsNum(expected)); - } - } -} } diff --git a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h index bbae06b66c6..d70b859ae1d 100644 --- a/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h +++ b/src/Storages/ObjectStorageQueue/ObjectStorageQueueTableMetadata.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include namespace DB @@ -16,29 +18,28 @@ class ReadBuffer; */ struct ObjectStorageQueueTableMetadata { - String format_name; - String columns; - String after_processing; - String mode; - UInt64 tracked_files_limit = 0; - UInt64 tracked_file_ttl_sec = 0; - UInt64 buckets = 0; - UInt64 processing_threads_num = 1; - String last_processed_path; + const String format_name; + const String columns; + const String after_processing; + const String mode; + const UInt64 tracked_files_limit; + const UInt64 tracked_file_ttl_sec; + const UInt64 buckets; + const UInt64 processing_threads_num; + const String last_processed_path; - ObjectStorageQueueTableMetadata() = default; ObjectStorageQueueTableMetadata( - const StorageObjectStorage::Configuration & configuration, const ObjectStorageQueueSettings & engine_settings, - const StorageInMemoryMetadata & storage_metadata); + const ColumnsDescription & columns_, + const std::string & format_); + + explicit ObjectStorageQueueTableMetadata(const Poco::JSON::Object::Ptr & json); - void read(const String & metadata_str); static ObjectStorageQueueTableMetadata parse(const String & metadata_str); String toString() const; void checkEquals(const ObjectStorageQueueTableMetadata & from_zk) const; - static void checkEquals(const ObjectStorageQueueSettings & current, const ObjectStorageQueueSettings & expected); private: void checkImmutableFieldsEquals(const ObjectStorageQueueTableMetadata & from_zk) const; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 9452ce81e9e..8f11836a11b 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -64,9 +64,7 @@ namespace void checkAndAdjustSettings( ObjectStorageQueueSettings & queue_settings, - ASTStorage * engine_args, - bool is_attach, - const LoggerPtr & log) + bool is_attach) { if (!is_attach && !queue_settings.mode.changed) { @@ -85,19 +83,12 @@ namespace "Setting `cleanup_interval_min_ms` ({}) must be less or equal to `cleanup_interval_max_ms` ({})", queue_settings.cleanup_interval_min_ms, queue_settings.cleanup_interval_max_ms); } - - if (!is_attach && !queue_settings.processing_threads_num.changed) - { - queue_settings.processing_threads_num = std::max(getNumberOfPhysicalCPUCores(), 16); - engine_args->settings->as()->changes.insertSetting( - "processing_threads_num", - queue_settings.processing_threads_num.value); - - LOG_TRACE(log, "Set `processing_threads_num` to {}", queue_settings.processing_threads_num); - } } - std::shared_ptr getQueueLog(const ObjectStoragePtr & storage, const ContextPtr & context, const ObjectStorageQueueSettings & table_settings) + std::shared_ptr getQueueLog( + const ObjectStoragePtr & storage, + const ContextPtr & context, + const ObjectStorageQueueSettings & table_settings) { const auto & settings = context->getSettingsRef(); switch (storage->getType()) @@ -117,7 +108,6 @@ namespace default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected object storage type: {}", storage->getType()); } - } } @@ -130,7 +120,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( const String & comment, ContextPtr context_, std::optional format_settings_, - ASTStorage * engine_args, + ASTStorage * /* engine_args */, LoadingStrictnessLevel mode) : IStorage(table_id_) , WithContext(context_) @@ -154,7 +144,7 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( throw Exception(ErrorCodes::BAD_QUERY_PARAMETER, "ObjectStorageQueue url must either end with '/' or contain globs"); } - checkAndAdjustSettings(*queue_settings, engine_args, mode > LoadingStrictnessLevel::CREATE, log); + checkAndAdjustSettings(*queue_settings, mode > LoadingStrictnessLevel::CREATE); object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); FormatFactory::instance().checkFormatName(configuration->format); @@ -173,21 +163,14 @@ StorageObjectStorageQueue::StorageObjectStorageQueue( setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); - task = getContext()->getSchedulePool().createTask("ObjectStorageQueueStreamingTask", [this] { threadFunc(); }); - /// Get metadata manager from ObjectStorageQueueMetadataFactory, - /// it will increase the ref count for the metadata object. - /// The ref count is decreased when StorageObjectStorageQueue::drop() method is called. - files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, *queue_settings); - try - { - files_metadata->initialize(configuration_, storage_metadata); - } - catch (...) - { - ObjectStorageQueueMetadataFactory::instance().remove(zk_path); - throw; - } + ObjectStorageQueueTableMetadata table_metadata(*queue_settings, storage_metadata.getColumns(), configuration_->format); + ObjectStorageQueueMetadata::syncWithKeeper(zk_path, table_metadata, *queue_settings, log); + + auto queue_metadata = std::make_unique(zk_path, std::move(table_metadata), *queue_settings); + files_metadata = ObjectStorageQueueMetadataFactory::instance().getOrCreate(zk_path, std::move(queue_metadata)); + + task = getContext()->getSchedulePool().createTask("ObjectStorageQueueStreamingTask", [this] { threadFunc(); }); } void StorageObjectStorageQueue::startup() diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index d242b6de4ec..ee99c472620 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -63,7 +63,6 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context /// current setting values, one needs to ask the components directly. std::unordered_map> changeable_settings = { {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, - {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}}, {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index 28dc43f350b..744d7139d16 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -23,7 +23,6 @@ class ITableFunctionCluster : public Base { public: String getName() const override = 0; - String getSignature() const override = 0; static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context) { @@ -46,7 +45,11 @@ protected: void parseArgumentsImpl(ASTs & args, const ContextPtr & context) override { if (args.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "The function {} should have arguments. The first argument must be the cluster name and the rest are the arguments of " + "corresponding table function", + getName()); /// Evaluate only first argument, everything else will be done Base class args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context); diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 1a58be4f75b..23e59494f61 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -57,7 +57,7 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr & context) { - if (args.empty() || args.size() > 4) + if (args.empty() || args.size() > getMaxNumberOfArguments()) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); for (auto & arg : args) diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index ba1b7d2bb3f..4c97507b8d1 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -15,6 +15,7 @@ class Context; class ITableFunctionFileLike : public ITableFunction { public: + static constexpr auto max_number_of_arguments = 4; static constexpr auto signature = " - filename\n" " - filename, format\n" " - filename, format, structure\n" @@ -32,7 +33,7 @@ public: NameSet getVirtualsToCheckBeforeUsingStructureHint() const override; - static size_t getMaxNumberOfArguments() { return 4; } + static size_t getMaxNumberOfArguments() { return max_number_of_arguments; } static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &); diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 3468e5c5007..6b923f93e75 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -23,83 +23,42 @@ struct AzureDefinition { static constexpr auto name = "azureBlobStorage"; static constexpr auto storage_type_name = "Azure"; - static constexpr auto signature = " - connection_string, container_name, blobpath\n" - " - connection_string, container_name, blobpath, structure \n" - " - connection_string, container_name, blobpath, format \n" - " - connection_string, container_name, blobpath, format, compression \n" - " - connection_string, container_name, blobpath, format, compression, structure \n" - " - storage_account_url, container_name, blobpath, account_name, account_key\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; - static constexpr auto max_number_of_arguments = 8; }; struct S3Definition { static constexpr auto name = "s3"; static constexpr auto storage_type_name = "S3"; - static constexpr auto signature = " - url\n" - " - url, format\n" - " - url, format, structure\n" - " - url, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key\n" - " - url, access_key_id, secret_access_key, session_token\n" - " - url, access_key_id, secret_access_key, format\n" - " - url, access_key_id, secret_access_key, session_token, format\n" - " - url, access_key_id, secret_access_key, format, structure\n" - " - url, access_key_id, secret_access_key, session_token, format, structure\n" - " - url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - static constexpr auto max_number_of_arguments = 8; }; struct GCSDefinition { static constexpr auto name = "gcs"; static constexpr auto storage_type_name = "GCS"; - static constexpr auto signature = S3Definition::signature; - static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct COSNDefinition { static constexpr auto name = "cosn"; static constexpr auto storage_type_name = "COSN"; - static constexpr auto signature = S3Definition::signature; - static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct OSSDefinition { static constexpr auto name = "oss"; static constexpr auto storage_type_name = "OSS"; - static constexpr auto signature = S3Definition::signature; - static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct HDFSDefinition { static constexpr auto name = "hdfs"; static constexpr auto storage_type_name = "HDFS"; - static constexpr auto signature = " - uri\n" - " - uri, format\n" - " - uri, format, structure\n" - " - uri, format, structure, compression_method\n"; - 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 @@ -107,14 +66,9 @@ class TableFunctionObjectStorage : public ITableFunction { public: static constexpr auto name = Definition::name; - static constexpr auto signature = Definition::signature; - - static size_t getMaxNumberOfArguments() { return Definition::max_number_of_arguments; } String getName() const override { return name; } - virtual String getSignature() const { return signature; } - bool hasStaticStructure() const override { return configuration->structure != "auto"; } bool needStructureHint() const override { return configuration->structure == "auto"; } @@ -142,7 +96,7 @@ public: const String & format, const ContextPtr & context) { - Configuration().addStructureAndFormatToArgs(args, structure, format, context); + Configuration().addStructureAndFormatToArgsIfNeeded(args, structure, format, context); } protected: diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index 296791b8bda..11e6c1fde82 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -19,40 +19,22 @@ struct AzureClusterDefinition { static constexpr auto name = "azureBlobStorageCluster"; static constexpr auto storage_type_name = "AzureBlobStorageCluster"; - static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; - static constexpr auto max_number_of_arguments = AzureDefinition::max_number_of_arguments + 1; }; struct S3ClusterDefinition { static constexpr auto name = "s3Cluster"; static constexpr auto storage_type_name = "S3Cluster"; - static constexpr auto signature = " - cluster, url\n" - " - cluster, url, format\n" - " - cluster, url, format, structure\n" - " - cluster, url, access_key_id, secret_access_key\n" - " - cluster, url, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, format\n" - " - cluster, url, access_key_id, secret_access_key, format, structure\n" - " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments + 1; }; struct HDFSClusterDefinition { static constexpr auto name = "hdfsCluster"; static constexpr auto storage_type_name = "HDFSCluster"; - static constexpr auto signature = " - cluster_name, uri\n" - " - cluster_name, uri, format\n" - " - cluster_name, uri, format, structure\n" - " - cluster_name, uri, format, structure, compression_method\n"; - static constexpr auto max_number_of_arguments = HDFSDefinition::max_number_of_arguments + 1; }; /** -* Class implementing s3/hdfs/azureBlobStorage)Cluster(...) table functions, +* Class implementing s3/hdfs/azureBlobStorageCluster(...) table functions, * which allow to process many files from S3/HDFS/Azure blob storage on a specific cluster. * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks * in file path and dispatch each file dynamically. @@ -64,10 +46,8 @@ class TableFunctionObjectStorageCluster : public ITableFunctionCluster; diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 436fc5b6bb2..3587405d95f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -473,7 +473,7 @@ class CI: JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - timeout=3600, + timeout=3 * 3600, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 5c051b093e0..d3c4ffb1e68 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -9,6 +9,7 @@ from urllib.parse import quote from unidiff import PatchSet # type: ignore from build_download_helper import get_gh_api +from ci_config import Labels from env_helper import ( GITHUB_EVENT_PATH, GITHUB_REPOSITORY, @@ -16,7 +17,6 @@ from env_helper import ( GITHUB_SERVER_URL, GITHUB_UPSTREAM_REPOSITORY, ) -from ci_config import Labels from get_robot_token import get_best_robot_token from github_helper import GitHub @@ -459,16 +459,18 @@ class PRInfo: sync_repo = gh.get_repo(GITHUB_REPOSITORY) sync_pr = sync_repo.get_pull(self.number) # Find the commit that is in both repos, upstream and cloud - sync_commits = sync_pr.get_commits().reversed - upstream_commits = upstream_pr.get_commits().reversed + # Do not ever use `reversed` here, otherwise the list of commits is not full + sync_commits = list(sync_pr.get_commits()) + upstream_commits = list(upstream_pr.get_commits()) # Github objects are compared by _url attribute. We can't compare them directly and # should compare commits by SHA1 upstream_shas = [c.sha for c in upstream_commits] logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) sync_shas = [c.sha for c in sync_commits] - logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) + logging.info("Commits in sync PR:\n %s", ", ".join(sync_shas)) - # find latest synced commit + # find latest synced commit, search from the latest + upstream_commits.reverse() last_synced_upstream_commit = None for commit in upstream_commits: if commit.sha in sync_shas: diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 6cb7d026417..75a180f356b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -916,6 +916,7 @@ class SettingsRandomizer: "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), "optimize_functions_to_subcolumns": lambda: random.randint(0, 1), + "parallel_replicas_local_plan": lambda: random.randint(0, 1), } @staticmethod diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index af26565d7e6..ff42b5828e3 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -115,10 +115,6 @@ false - - 127.0.0.1 - 9000 - 127.0.0.2 9000 @@ -147,6 +143,10 @@ 127.0.0.8 9000 + + 127.0.0.1 + 9000 + 127.0.0.9 9000 diff --git a/tests/integration/test_broken_projections/test.py b/tests/integration/test_broken_projections/test.py index 578ff42369c..d161f3fba78 100644 --- a/tests/integration/test_broken_projections/test.py +++ b/tests/integration/test_broken_projections/test.py @@ -735,11 +735,14 @@ def test_mutation_with_broken_projection(cluster): f"ALTER TABLE {table_name} DELETE WHERE _part == 'all_0_0_0_4' SETTINGS mutations_sync = 1" ) + parts = get_parts(node, table_name) # All parts changes because this is how alter delete works, # but all parts apart from the first have only hardlinks to files in previous part. - assert ["all_0_0_0_5", "all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == get_parts( - node, table_name - ) or ["all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == get_parts(node, table_name) + assert ["all_0_0_0_5", "all_1_1_0_5", "all_2_2_0_5", "all_3_3_0_5"] == parts or [ + "all_1_1_0_5", + "all_2_2_0_5", + "all_3_3_0_5", + ] == parts # Still broken because it was hardlinked. broken = get_broken_projections_info(node, table_name) @@ -752,11 +755,13 @@ def test_mutation_with_broken_projection(cluster): f"ALTER TABLE {table_name} DELETE WHERE c == 13 SETTINGS mutations_sync = 1" ) - assert ["all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == get_parts( - node, table_name - ) or ["all_0_0_0_6", "all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == get_parts( - node, table_name - ) + parts = get_parts(node, table_name) + assert ["all_1_1_0_6", "all_2_2_0_6", "all_3_3_0_6"] == parts or [ + "all_0_0_0_6", + "all_1_1_0_6", + "all_2_2_0_6", + "all_3_3_0_6", + ] == parts # Not broken anymore. assert not get_broken_projections_info(node, table_name) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 56ea5c8846a..a4b2399e117 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -13,8 +13,20 @@ def started_cluster(): main_configs=["config.xml"], with_minio=True, ) - cluster.start() + + # local disk requires its `path` directory to exist. + # the two paths below belong to `test1` and `test2` disks + node = cluster.instances["disks_app_test"] + for path in ["path1", "path2"]: + node.exec_in_container( + [ + "bash", + "-c", + f"mkdir -p /var/lib/clickhouse/{path}", + ] + ) + yield cluster finally: diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml index 25ececea3e8..e71b93379d0 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml @@ -16,7 +16,7 @@ az-zoo2 1 - 20000000 + 200000000 10000 diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml index 81e343b77c9..cf4a4686f2c 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml @@ -13,7 +13,7 @@ 2181 3 - 20000000 + 200000000 10000 diff --git a/tests/integration/test_memory_limit/test.py b/tests/integration/test_memory_limit/test.py index 6d6745711da..db68a38c1b1 100644 --- a/tests/integration/test_memory_limit/test.py +++ b/tests/integration/test_memory_limit/test.py @@ -13,7 +13,6 @@ node = cluster.add_instance( "configs/async_metrics_no.xml", ], mem_limit="4g", - env_variables={"MALLOC_CONF": "dirty_decay_ms:0"}, ) diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index 62d4b005d94..5ae4cc7d882 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -49,5 +49,6 @@ def test_skip_all_replicas( "max_parallel_replicas": max_parallel_replicas, "cluster_for_parallel_replicas": cluster_name, "skip_unavailable_shards": skip_unavailable_shards, + "parallel_replicas_local_plan": 0, }, ) diff --git a/tests/integration/test_restore_external_engines/test.py b/tests/integration/test_restore_external_engines/test.py index cf189f2a6ed..a975db05020 100644 --- a/tests/integration/test_restore_external_engines/test.py +++ b/tests/integration/test_restore_external_engines/test.py @@ -70,6 +70,12 @@ def get_mysql_conn(cluster): def fill_tables(cluster, dbname): fill_nodes(nodes, dbname) + node1.query( + f"""CREATE TABLE {dbname}.example_s3_engine_table (name String, value UInt32) +ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip') +SETTINGS input_format_with_names_use_header = 0""" + ) + conn = get_mysql_conn(cluster) with conn.cursor() as cursor: @@ -136,6 +142,7 @@ def test_restore_table(start_cluster): node2.query(f"BACKUP DATABASE replicated TO {backup_name}") + node2.query("DROP TABLE replicated.example_s3_engine_table") node2.query("DROP TABLE replicated.mysql_schema_inference_engine") node2.query("DROP TABLE replicated.mysql_schema_inference_function") @@ -149,6 +156,13 @@ def test_restore_table(start_cluster): ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated") + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated' and name = 'example_s3_engine_table'" + ) + == "S3\n" + ) + assert ( node1.query( "SELECT count(), sum(id) FROM replicated.mysql_schema_inference_engine" @@ -175,6 +189,7 @@ def test_restore_table_null(start_cluster): node2.query(f"BACKUP DATABASE replicated2 TO {backup_name}") + node2.query("DROP TABLE replicated2.example_s3_engine_table") node2.query("DROP TABLE replicated2.mysql_schema_inference_engine") node2.query("DROP TABLE replicated2.mysql_schema_inference_function") @@ -188,6 +203,13 @@ def test_restore_table_null(start_cluster): ) node1.query(f"SYSTEM SYNC DATABASE REPLICA replicated2") + assert ( + node1.query( + "SELECT engine FROM system.tables where database = 'replicated2' and name = 'example_s3_engine_table'" + ) + == "Null\n" + ) + assert ( node1.query( "SELECT count(), sum(id) FROM replicated2.mysql_schema_inference_engine" diff --git a/tests/integration/test_s3_cluster/configs/named_collections.xml b/tests/integration/test_s3_cluster/configs/named_collections.xml index 64d1bd98df2..2d3a69a8c38 100644 --- a/tests/integration/test_s3_cluster/configs/named_collections.xml +++ b/tests/integration/test_s3_cluster/configs/named_collections.xml @@ -6,5 +6,12 @@ minio123 CSV> + + http://minio1:9001/root/data/data{1,2,3} + minio + minio123 + JSONEachRow> + id UInt32, date Date DEFAULT 18262 + diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 03919ee6a4d..c31851fdfe9 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -459,3 +459,51 @@ def test_cluster_format_detection(started_cluster): ) assert result == expected_result + + +def test_cluster_default_expression(started_cluster): + node = started_cluster.instances["s0_0_0"] + + node.query( + "insert into function s3('http://minio1:9001/root/data/data1', 'minio', 'minio123', JSONEachRow) select 1 as id settings s3_truncate_on_insert=1" + ) + node.query( + "insert into function s3('http://minio1:9001/root/data/data2', 'minio', 'minio123', JSONEachRow) select * from numbers(0) settings s3_truncate_on_insert=1" + ) + node.query( + "insert into function s3('http://minio1:9001/root/data/data3', 'minio', 'minio123', JSONEachRow) select 2 as id settings s3_truncate_on_insert=1" + ) + + expected_result = node.query( + "SELECT * FROM s3('http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262') order by id" + ) + + result = node.query( + "SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262') order by id" + ) + + assert result == expected_result + + result = node.query( + "SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'auto', 'id UInt32, date Date DEFAULT 18262') order by id" + ) + + assert result == expected_result + + result = node.query( + "SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'JSONEachRow', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id" + ) + + assert result == expected_result + + result = node.query( + "SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/data{1,2,3}', 'minio', 'minio123', 'auto', 'id UInt32, date Date DEFAULT 18262', 'auto') order by id" + ) + + assert result == expected_result + + result = node.query( + "SELECT * FROM s3Cluster(cluster_simple, test_s3_with_default) order by id" + ) + + assert result == expected_result diff --git a/tests/integration/test_ssl_cert_authentication/certs/client5-cert.pem b/tests/integration/test_ssl_cert_authentication/certs/client5-cert.pem new file mode 100644 index 00000000000..b17baa62262 --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client5-cert.pem @@ -0,0 +1,33 @@ +-----BEGIN CERTIFICATE----- +MIIFojCCA4qgAwIBAgIUBfEMZ1Z/4weV13ryVA9qyNTPJHEwDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwOTAy +MTYwODI0WhcNMzQwODMxMTYwODI0WjBXMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMRAw +DgYDVQQDDAdjbGllbnQ1MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA +zivZ5IrYyoJeDX0Zbl/cl8rKE0LbmtD+QKZvQXHD+gstXWxPOEFZVxO3BuqmvBZ1 +MaYNyPYA9JyyP+pO9cE8RrTk3w9xMhv8dYWJQK7na9W9RTSXs8xhPwiEm4XuVgqv +GfK/EvdxbFMhgzExOR333TkmXPxrMm5xPWwV3RsTCjNVW7nmdPHXmchuTO7lQtww +6fETqc1Zqv8TO2x/uGZiwAzNYbueWHyzL4Y1UJ7D0mnNNgJvvxtcrzBTlGLLhJ5o +0+zVQLaOnac0WVk0wDhVaxd/gR4bYN3ixvuFbbOaTroFVTMVme196G2FkJI/05Pn +D68r1yUpvuNDjkBbuBO43PlsFKLhPU5twfu4TesEb2WZ0VsNxn8Hc0Ne02WwHsHa +Fi1N0oXvVIeGvvzevuTkjPbh2aCaQX9qbhLXjhgXNFAXQq+qI8ECCWC4LruPL1Es ++ZM2thQAg4k8GY4U9Q8zX55Ut7t9T771QbHFOAupuTgZJSz0jC8JId0m7NOOqtvu +4w/GqefCt9bp+wlQDXVOgi+S3GqKM1OuIbs5XQJtNfi6l684ptAFHSHMHXvfGLHI +MlKdobjNqEFVK9qO/3YvnTUyRyaUA61aHxraXWBwvcUByXwXdgbuz/M5XAi6fr0V +Trw9iUSviDvu4HwDo8PCw9i8xYFtkr2k1kurWEsFZMcCAwEAAaNrMGkwJwYDVR0R +BCAwHoYcc3BpZmZlOi8vYmFyLmNvbS9mb28vYm9vL2ZhcjAdBgNVHQ4EFgQUENIS +25yehLQrlL8vC+DpkNE7urowHwYDVR0jBBgwFoAUiSo9XUmDdI1vjLtMRUK6G2RZ +kKUwDQYJKoZIhvcNAQELBQADggIBAADH/LINEwB1WUQ2Q/aHiNv1ZyJ+ifgs9ewp +/t0uwS+53ctcmJ6Sqeni3/UIdLYjpcdBbl1XpqTYuGF28C2YZrWKFB0HaOiH6D8B +zcGGAkIKFqnhcJxyl37je+scZ8Wk9b04L+X+miN6cgIWm6rQezDF3rs1xvAVBqTM +YPIk6sBIgHNJy4D3S5KdhqNV0/8KY6T65nGFdEq064qOk8HvS6DyYOs22AitCD+L +gcWGJHJ3BfNASbRrT25zb1HLUIFFbFIGaPFd9GbiU5hGb9MgUzX44q+WdXoEa59a +6y9ZcidjEqAGP/FMz16D831YpqRBherZ09ztWXeTfv4NxauisLuoqpOr7CmpQ+Ct +O5t0cUHILeNBFR7rdMOmDawpEcOSGqcJHdPH4SjP/LtgQODWiNys19Yp5afbM5Lz +IjLjq1wAHVtSvPHjRhnZSq0SiU1XlDmu1Em3HbFe5RmqL/lcLe7/U10ddngADG7E +XgPE0jcvl7rYASqYuTbKd6Q53QYx0K7xc1n8mIRJuAofPwl6Yns/ytvw0+E9TBS1 +oGb7j6V/k+Xd77dfJ6fckJXPg7Fm3GPO1ax7FNU51sCrvAHsMZhiWQa6pZzBEORM +4yI+DSFyskyWXCPth9r3UqHQXzX86LRkyDWg9l6v3NWRSI1j/e7dZds/U/sg2maq +css4A+kM +-----END CERTIFICATE----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/client5-ext.cnf b/tests/integration/test_ssl_cert_authentication/certs/client5-ext.cnf new file mode 100644 index 00000000000..8cb20e70290 --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client5-ext.cnf @@ -0,0 +1 @@ +subjectAltName=URI:spiffe://bar.com/foo/boo/far diff --git a/tests/integration/test_ssl_cert_authentication/certs/client5-key.pem b/tests/integration/test_ssl_cert_authentication/certs/client5-key.pem new file mode 100644 index 00000000000..aa65de6e26c --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client5-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQDOK9nkitjKgl4N +fRluX9yXysoTQtua0P5Apm9BccP6Cy1dbE84QVlXE7cG6qa8FnUxpg3I9gD0nLI/ +6k71wTxGtOTfD3EyG/x1hYlArudr1b1FNJezzGE/CISbhe5WCq8Z8r8S93FsUyGD +MTE5HffdOSZc/GsybnE9bBXdGxMKM1VbueZ08deZyG5M7uVC3DDp8ROpzVmq/xM7 +bH+4ZmLADM1hu55YfLMvhjVQnsPSac02Am+/G1yvMFOUYsuEnmjT7NVAto6dpzRZ +WTTAOFVrF3+BHhtg3eLG+4Vts5pOugVVMxWZ7X3obYWQkj/Tk+cPryvXJSm+40OO +QFu4E7jc+WwUouE9Tm3B+7hN6wRvZZnRWw3GfwdzQ17TZbAewdoWLU3She9Uh4a+ +/N6+5OSM9uHZoJpBf2puEteOGBc0UBdCr6ojwQIJYLguu48vUSz5kza2FACDiTwZ +jhT1DzNfnlS3u31PvvVBscU4C6m5OBklLPSMLwkh3Sbs046q2+7jD8ap58K31un7 +CVANdU6CL5LcaoozU64huzldAm01+LqXrzim0AUdIcwde98YscgyUp2huM2oQVUr +2o7/di+dNTJHJpQDrVofGtpdYHC9xQHJfBd2Bu7P8zlcCLp+vRVOvD2JRK+IO+7g +fAOjw8LD2LzFgW2SvaTWS6tYSwVkxwIDAQABAoICAAxYkuU9LUs/MEulfKLf6bFL +SvuJSuZD8p3DebEVE3PDPJW2GS5Yt3Vf8FthLNfhjmYJOBUbKiZ7xa7PJKhRyH7B +cV0sKw6hn2YYN4ZgkuohtAJTamwxZndkVrCLfiC35hFrOGb121xtw/l7tiNh+IhO +Vk5GLVMpSu2vLcX+S48WUNu7z6GcI9mJJ5XCzobI8cLolVFbQy3atXefASz1MJ+o +hJoAJrTMztwfMM0hnL9aZ5f+4Fs008GH5tFhcyd/Zu6d5Y1JAVROgXOCRIboOLE/ +iHqeZ2xPDTf2MuDFWw2w1QXrl7UOhDYpbNCh+WF9n14QktMA3jWYnWCE2Rk2USEK +2QhsScNgA7dY3S3DbSK1ZiyZEgbvdq6EYy5I46jNXqRrfKfuD7J04dtO1pmBWZy3 +a765FMiejbHrC1xrKtSnvGj1oJ23P3TRiEL6tx3QF4bAulWE62ULBfDgqr56pPbt +KOfPinEFa/14+B4a21iFzsoA6SWPt+i/k00RyRHw4X7acTd8lsavHcT8PGY4wMLX +cNEombZn4h/0oCqAmBtoXYwctcyQwzhJNzAp1GFMG2gdBPhXUsWcwwjNNzeaewCq +BO0YO/DFgj1qTHZwswTfppJMF+BTE/sol4eaqOV1TYV48OfYTdo+6dqfH2JkaJyy +PVJAoZtKKPfDRlrHMrE9AoIBAQD1oZMIi4Nm4TpdOGsU5O2keDW6DXdoxpOI6u9H +YqWZlSZ57q2u+8JkoVJhH/jq4l1QuXtUObpQ9DYOpPwPkt+ZblUut/ZrmMo+gs81 +RtIv4EGmgFmtrXgmkYGsoNrIu4/ayBPDwGz7Z+gEta+gxhYbxSZgb2ZOZzH/c2my +3CvfgFiYyv/cfVvynjprRZoxowQvJoMCEEA9REZdO5T51lshQFcrpFU2ucQNKV60 +DELV6uJzhL0EDvg9IqP1CxZhdCsTifE/SNVepNWXFegFGVsD/vxmXVxJYevSDQZY +SvGWVcgyuQA8Gdze6y3jOUlzCQ3v7D63PgAPS+yrGXLd0Vz9AoIBAQDW39oRb1n2 +ves41cEz7qPNiGmc+zhzyZE30HFnl/RxREPfhzvifTUk3YTUPdJGpcgf+sMoPQ/R +plx/ZrFPTyV7U+svf/eoPlpNnVnZ1m+C/nN7AjVwq04GRHKYD3lTD7KpoYQ0ICGO +z9xFiGbK4VRNIiNpyGc3IZBmrwB2wsGn1L4yfFZly3Th4qiDmNpR29rgteQYWSNJ +hSqsCFcqogJfppP+QKRGf00uHDa1AGQhB2eFykyAniQw52FcdpUdkGqYiR1vdir7 +5XckNixq7yyRflo8CftHybyHipTBZVXBzdvDpY5mxANPcJlSPhJxqruxK9FzNYv6 +OiDlFnFPwNYTAoIBAQDLByRbWcXhEaWGTA3mlVsAKQRvppXemtRzxYzGOnmaure0 +7V3OVvZU6ysGcMD0lOqAwO95rMWeCF1uOVzHGqWLUrFCDni2cvIoPUM8TC+LtxdM +oqZ9cfbwGjFPGR398Vp0ghAkKzdpjncu/WYPw+ueRJT1/N5ZS979wM3LM0VoVdPl +Z1WZUFzh63tqE3viL1ZUCscau4f9nCN1CLyYzV9B2sayogB1XZL5Ngq7U68i1iUb +SspZQm5ZSfqvntx+7OB2I/yuTGtg8WpCma7QGCcs5GSHz/9qAHBFjNKDxF5v3rO9 +iUDybIYuE8I4IK/fT8qfV2x3Vd4CxsT2n/Bb/KOZAoIBAQCyIwrqgWIYCQNFLxDp +OOJbtse3+R2oAGd/2jLqKzPf4zRR0A95iUvRsEQ6oLX/E9mBiabZaSYALOdqR3yf +v4HXaI8F5hUvSeIbjmO7EOeJteGPDRm4uowI7h4CqnFuxwjbUKgFfs3TU8fNbXOq +pnv5JmAthpLrRcwtFNBRpMxfkyPLPwFxiubvjbUexE3ap2Yh/SmIdf2nKdtim9eH +5KALJFJ06qpGN6uImqNQE27vYvAUHs6lonVmhaxVt4mP5PY6VxIsIc8o3eeUAcV5 +MafFRvcP50aHEVxXEjCY1KXv8fZLkKkp0T7dUQALCqLH0T+hdi5CURYm8KHsylpO +QBQ5AoIBADXNlx5bXEU2lW21wyWG4v3gqvmnDgQG66RXaHC0Rgpn6BW7pose40fv +I82yWviabk7wlvaQgXllzZ5vGL43/8pp+wFpRUl5nX8P1ZA8pRkRArm1vLxQRjkM +90j0M/XhTnC8eC4qKsQx8dPUH0SPkHJAd2tWcP9Q3635zETcINkpsNbsB/QNGBZ1 +JefHA/GNZZBrjFMOnvzEbEbqBdhwlZNRlZxVq/svvNzzv12jFQpM7HVy95bATiZ8 +x7SQlDxCWOWApd9VpM83JnPWCfu0Yi/HHPJd6YdueMhRSvUpLs1sD8fs9bFQ7+OL +erqwgB4b3yDlXijY75pPDxdPf/0qA3Q= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/client5-req.pem b/tests/integration/test_ssl_cert_authentication/certs/client5-req.pem new file mode 100644 index 00000000000..6ad15243bac --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client5-req.pem @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE REQUEST----- +MIIEnDCCAoQCAQAwVzELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx +ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEQMA4GA1UEAwwHY2xp +ZW50NTCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAM4r2eSK2MqCXg19 +GW5f3JfKyhNC25rQ/kCmb0Fxw/oLLV1sTzhBWVcTtwbqprwWdTGmDcj2APScsj/q +TvXBPEa05N8PcTIb/HWFiUCu52vVvUU0l7PMYT8IhJuF7lYKrxnyvxL3cWxTIYMx +MTkd9905Jlz8azJucT1sFd0bEwozVVu55nTx15nIbkzu5ULcMOnxE6nNWar/Ezts +f7hmYsAMzWG7nlh8sy+GNVCew9JpzTYCb78bXK8wU5Riy4SeaNPs1UC2jp2nNFlZ +NMA4VWsXf4EeG2Dd4sb7hW2zmk66BVUzFZntfehthZCSP9OT5w+vK9clKb7jQ45A +W7gTuNz5bBSi4T1ObcH7uE3rBG9lmdFbDcZ/B3NDXtNlsB7B2hYtTdKF71SHhr78 +3r7k5Iz24dmgmkF/am4S144YFzRQF0KvqiPBAglguC67jy9RLPmTNrYUAIOJPBmO +FPUPM1+eVLe7fU++9UGxxTgLqbk4GSUs9IwvCSHdJuzTjqrb7uMPxqnnwrfW6fsJ +UA11ToIvktxqijNTriG7OV0CbTX4upevOKbQBR0hzB173xixyDJSnaG4zahBVSva +jv92L501MkcmlAOtWh8a2l1gcL3FAcl8F3YG7s/zOVwIun69FU68PYlEr4g77uB8 +A6PDwsPYvMWBbZK9pNZLq1hLBWTHAgMBAAGgADANBgkqhkiG9w0BAQsFAAOCAgEA +SGJaGaSkfsUBSqPbkbtyE2ndHIY8QMPvJAMB7pPbA/MlX39oO2U1AraFatN/Xzpd +0uQA2m0UsXF2+ScrG4rd14YkU4wpcT2lIplvFFzUkwfYLATRYrFV0FCru1n28+Ve +TyUG9/tlf/jnNZwWtB5AEbCWmzXSjXUuwXdn2oQ0z+3dxdSnpOrDkzyYeocIMnMJ +pG0qdBd+hJrK6snJNli6EfsmmykyCSLorFpnkm2uKUwdTAD2/MlxYdutjk7tqKw5 +wpYi6Wqt/euyQ94Ri4aICZMBFk5+zDdaEOKVQZ0aUT8RWt/aD2ksbTrclDICOe84 +iG+Nf/CyNqSNqsUugCSCxN3UUZgLKEyJb8Pz4N/nFFRBb1aiZ/5YVRTYtZ+k4tLb +s2exxqVpDg3M5G5bk0iJ8DBEjO/yKwobc7HAtnTAEIs0HGord3yzg3F0Y+5ecQAg +9ESOptz33EBkTHxpBmt0D0ACa4CTghrf/Id8imNtdGU9i7MeEba+iUVAUP8VfhtL +FJPRR8aVaKaLc9uCAiiHuRc+J0EHAwTOKKTK3Z1mkYO00kMCiRcru8/H6ibkrkV7 +kRL6NvAc7CsEzSDPkFKAZhQ4p6AcfNC5yRiNLG9JB/wQgBg8v23Uwtk74gOXIaQN +WUjwzdYOljdcFOzmoDMzyfIFwSXcO3dmmjqOwh2HNw0= +-----END CERTIFICATE REQUEST----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/client6-cert.pem b/tests/integration/test_ssl_cert_authentication/certs/client6-cert.pem new file mode 100644 index 00000000000..b83c380a3bc --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client6-cert.pem @@ -0,0 +1,33 @@ +-----BEGIN CERTIFICATE----- +MIIFojCCA4qgAwIBAgIUBfEMZ1Z/4weV13ryVA9qyNTPJHIwDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwOTAy +MTYwODM0WhcNMzQwODMxMTYwODM0WjBXMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMRAw +DgYDVQQDDAdjbGllbnQ2MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA +uz+3VyFhBD6K68mkDM0LJfRfGbuD8tSmHhtO6+XQzhIUiANW+A1WdyD+zWky0QsS +vl/2Ds1PmjU7659pkSD8Sidjdz3/TP0eO49nHinLjRQa2Oawk6PCjLIcpJ+A5eGb +Hno/oiQBMCAPpVh3sex+5yPiUQN62cKRWryv2JJqJmEgvpjC92SaIjf9M3mCsxMf +an0CZj6hNcorxXQNnF4JZRPQ4mMgBqgFS5Oz1YujHiBUN9ZoIGmS3HZ9LYl5lL7o +GxNXluyIrIw1kiyQrx+wJFdYwTBeyCqw8wmf993tHRmvpJ2ZFVXTbYqpj2Qkom+J +EpCRBqCmK+/uSgLOAicS/wR2eiYXkw2rYDIQ6yCyeW7nnaFVWNAFNSdAW1Jdr+Z2 +fKMolhYAcpeMQAYN5wTqJBF9dDwMxbOcdSh1wPZcenuO2q5cMJcn3qt3+SigNKcQ +BvqtZ54GNGq9h24f11+cqco80v4WYxrmXu8bXQfgtX07UVvqkjbG7O4HtbGDvOlw +KO7d1kOj4XUJdZbz9g5kaGufN4hlBs9JzYNOZdywNwBiPxHHE71Ht2ihfbBKSl9x +4Zse5YhPqToacWd5FRW+If5EpHkxxW+f4e61S2/8fnn5yHpMX22HXNBmOUR3vBrl +xfkje22ZuEf5NfB95aEaRZABmXQaHKdAVQeaAT9TvPMCAwEAAaNrMGkwJwYDVR0R +BCAwHoYcc3BpZmZlOi8vYmFyLmNvbS9mb28vYmF6L2ZhcjAdBgNVHQ4EFgQU0ieN +0CTYiMz7HJs9OH2U4imSzN0wHwYDVR0jBBgwFoAUiSo9XUmDdI1vjLtMRUK6G2RZ +kKUwDQYJKoZIhvcNAQELBQADggIBAELD77yWem26DIWG2Mi6q9KvLb/d8bOFN4mg +SYYekdEryyYCFhbb0P9T7GKj4KxNxkZaGgwXl+PM8SH7FhqemRCwCtHMVQiWvC1x +XLCrGiep7Dx+vvKHoCdapQp3uRfsZMHSZbsHJgQRFRt/jgP3jFcrLAa9tD8Yza4y +lBhh8pUROC0edlqom+BkLYA6A7FWA89x/NZL6wiyPKqRh2SuACszrlQdj4oqdgIF +pAILjDy4fLaCGGUgu9kHonV0xTjqSdFXiSeImSkePOvGWrh2B5GqRbanPHDSR5VD +pIhKZPpnvUBNhzoAMv1RFbgNs9m3sYqYChxb8IOGiY/3EG4rgygVez2yu16v5WMU +PIWwSv3N+oceG085dHxhn4TcDamdFrQHssp7GJBbyGEINHQRiZ4cu3Vz2cVNYcKD +iFJZ8vVwU7IZOEQeww6DT+gL+wqSgTyDvEXQNbYupFYPZXSBIaygH4eHa+PqPnNL +DJTpPAlwNRB2+eL3bZxvNAfwqIY6xgwnLBr1QrosmfMKgkswkg8gVoIpIRGACzk2 +iY818Jn+IG/M/aPF3p5dTsOqH3bQmz4ZpoLB2dytqkNDGKSAPPGOpe4MDyx0prCH +GWDjEjn4xT9AjAJVpTWJCENPzFzwA7byApuZwLTy/5ZBbNJf1K/JwsqXUhHKENb2 +NzMKvQCT +-----END CERTIFICATE----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/client6-ext.cnf b/tests/integration/test_ssl_cert_authentication/certs/client6-ext.cnf new file mode 100644 index 00000000000..d421f4198b4 --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client6-ext.cnf @@ -0,0 +1 @@ +subjectAltName=URI:spiffe://bar.com/foo/baz/far diff --git a/tests/integration/test_ssl_cert_authentication/certs/client6-key.pem b/tests/integration/test_ssl_cert_authentication/certs/client6-key.pem new file mode 100644 index 00000000000..e56fed5dddf --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client6-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQC7P7dXIWEEPorr +yaQMzQsl9F8Zu4Py1KYeG07r5dDOEhSIA1b4DVZ3IP7NaTLRCxK+X/YOzU+aNTvr +n2mRIPxKJ2N3Pf9M/R47j2ceKcuNFBrY5rCTo8KMshykn4Dl4Zseej+iJAEwIA+l +WHex7H7nI+JRA3rZwpFavK/YkmomYSC+mML3ZJoiN/0zeYKzEx9qfQJmPqE1yivF +dA2cXgllE9DiYyAGqAVLk7PVi6MeIFQ31mggaZLcdn0tiXmUvugbE1eW7IisjDWS +LJCvH7AkV1jBMF7IKrDzCZ/33e0dGa+knZkVVdNtiqmPZCSib4kSkJEGoKYr7+5K +As4CJxL/BHZ6JheTDatgMhDrILJ5buedoVVY0AU1J0BbUl2v5nZ8oyiWFgByl4xA +Bg3nBOokEX10PAzFs5x1KHXA9lx6e47arlwwlyfeq3f5KKA0pxAG+q1nngY0ar2H +bh/XX5ypyjzS/hZjGuZe7xtdB+C1fTtRW+qSNsbs7ge1sYO86XAo7t3WQ6PhdQl1 +lvP2DmRoa583iGUGz0nNg05l3LA3AGI/EccTvUe3aKF9sEpKX3Hhmx7liE+pOhpx +Z3kVFb4h/kSkeTHFb5/h7rVLb/x+efnIekxfbYdc0GY5RHe8GuXF+SN7bZm4R/k1 +8H3loRpFkAGZdBocp0BVB5oBP1O88wIDAQABAoICACymDcaV6/dCHogIpbhzsAPV +2FNfdiAR+qZVJqVzQC3S+9hOy48MRyDS2k8KlZZpCIgig56V8DQ6G1acxWRYtC/O +YpZNTzIBbRMQp6r2llXGhHxRzar2sm4wDkpmyiqGeCF1TvUPlsTt8C0iAjCHzt64 +nL9qkAGatmQnd9qxVuRd5pvr+xlYgLRGG3cJs1OV7LjMpCTTLEeSNIu5l4FAnwbe +CcHhlwJfUBvsVUZHLJcDaHGEC4InCmDNVB3mmPoR53CFVS5kqlZSfHelbr6DVNHl +jgFK0l7yZw0cr2tAHMkClfIvrg/7ThXhqXrKiz28ULf/hsVIyzbQ2EYHky3KOWny +04O7/NnOkVHs+XUyNC4sv9nkcd9ntKkvvUPPK0U6vbW7IasC3jCh5LMyZjHYwgmK +hzxEBZSyutKWn3RWncarwQ/1Vbq3HjbkeVTipXAa7Bny17wiAeZMZ2GqQZ9VcNQ3 +YJWDgxS5shwcEo+71sC4o2HjmWKcPujmt84XcWc6yphPbCpbwwswaQD5MaZxeDgZ +OUhF9LfslzNrwwoZTFz/Qhy3TOshF7BIbUdQnWLMNdAb9ccby/0WgOmfD6V4t99N +ksb2nWgvvK3isycs6HHVP/fgv+yM9cKGs66JoH2Jm+SInUtpR5Gv1aGeV97/9WFd +JuiHtHQIty+8n6GDTscJAoIBAQDuHCBeZ+pVTyW6wxjd4OD2keuDjM3Z7X/UgCoJ +kR87Dwjd8SHEw8QaH8vvGYBNktFu3KbQ1TV2OR8gAdnwlHeI5V2/nIVX0UBjQM9X +GC3cmzsMOBAem0nuYXZG9yvawwPUdZ18fQc2wAs4GqL4uKaOeuCefNyK5wKfvX7M +sA49D45gvLUhpfkTeM8HK9UQwMfMg2fFBzZifqTIG4OGkkAeEY+rkJTUxnvTuuFU +dkXXF8Qe+pSPkbQVQYYRRO9Wk0i16R6VaYrl3vvi72w2gEw7iQya0A1bHZe3s7vv +jQuz8h954kcgLYCqsOm/mj3t654jrjW1Z5yRjznTUJKrKMh3AoIBAQDJUVCp2Frm +NgzrZXD1QrkJ1qCRBHyVu7FikXqNszc9lLD5y8YWRnhDtGruOQ3DYjpuD/DMrO2P ++iBTambM3mJt6FE8TkXHyMzLoJ/I8SMLMbLNdDpsj8D8RlftwIESiNu9DQfMle5l +8jxZ7R7usio8HysVm5u6fsSmYVUZF+sWLLAUqote4HQxdvDup9A1q7onVZUYfKnK +mCVKqfdqFDqMKTSHOngxA5wzQWdMqdgLxiYKPYbkNsJ3dhXQwJjfbyDQq4X/foec +0wG91/WqsLiMOZLsQBiGMgOq85IqGBByl51QnkT8drPEJsXX6UCHjQ7AYHe0U+pe +JTa6nMfk2AplAoIBAQDemJa+CuFrvSEldowBqOBGQeXtRi2PBNNTAjnKVcvfd0+v +xGPwQZ9LWNxaevmSC6YUwNyGMDvZeM8bKf/nQ7R32w0IOzgA/9L0trrezfDZ4piR +9LtFEaFM4/ohn6J00Yj8DrQak/uxeFlEqsoeQOkcG81u/IVkqU+vrAQlMJUvCiLt +VpzyhunSBUdtidGW5tIh49qXvAVgkMpVdDtCC+k68unf1rr8K03Jg1RxlFv4F/S1 +jUZi7TBwCqBd9pbU1b3MqdF4loFOImZSIceFL+2UXqvU8pj5zDFwf+s6qB3/rGM2 +m44oi8FUUS1EfNpWWMWuz4bQPruE7GN/pDxpHChDAoIBAGZF5yLCBTvazalwuXf/ +051J6fyCOQCgf7dBjO8b0r54IYhlm1aJqmNK7t/jIDjYWDK96mkwmOeB/JYkAeGm +QH7xfQOUCCM8wb3Y9hPRXSo8r0ds+plYVoRTACyKlYfi+y8JxaKLsLcd3scYjZRZ +8tbkRrENgom2CRU1cVP0MLvtK+7zzSYABUdz02aK3l3FxiZhdgMgoemIbbmGq2i6 +qhu2ezcP3yuXV+06Fs59ooIowf6Fz1d08kpaNS7+CSvJevRHjyWDFEX5dHMLmYSD +jt2+CgP3c/4IvpBEeUblPsXYfFUOcqGHdD/8KppLpzq10H6vA2EbGH+HjEw1hd+e +WXkCggEBAIq+MHXpbipKp7lsxsp/moeNXUojLfiBcQLU+zauWBdP3kTJmDd7D3Vy +MoN4zpvhboue70RP1ZZG3yp8s9yR62XrO+44DrKKIhVitpa1IPSavFovFVl/vx1H +F6PuZvACBF7mRZa23h9XSizvdfycEDf3rZidLG/Y9IarrLF3HYjClBktJTtm0YaU +QnmTSUdNvRBCs6/BGQNXY76VB5AHNjeLjjuLLbxYF5XsHlsdacgdMv2ShmIcibrT +nSMK3RdRdRt1Nu1k6v7MkGcaSky7zJIeu6+VftA/7bVKBcnvb+iGZSUSk2QRTCGW +nT+c65hmPp61jcBOgCF3CntcIw4eEDc= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/client6-req.pem b/tests/integration/test_ssl_cert_authentication/certs/client6-req.pem new file mode 100644 index 00000000000..1723c8c7273 --- /dev/null +++ b/tests/integration/test_ssl_cert_authentication/certs/client6-req.pem @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE REQUEST----- +MIIEnDCCAoQCAQAwVzELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx +ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEQMA4GA1UEAwwHY2xp +ZW50NjCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALs/t1chYQQ+iuvJ +pAzNCyX0Xxm7g/LUph4bTuvl0M4SFIgDVvgNVncg/s1pMtELEr5f9g7NT5o1O+uf +aZEg/EonY3c9/0z9HjuPZx4py40UGtjmsJOjwoyyHKSfgOXhmx56P6IkATAgD6VY +d7Hsfucj4lEDetnCkVq8r9iSaiZhIL6YwvdkmiI3/TN5grMTH2p9AmY+oTXKK8V0 +DZxeCWUT0OJjIAaoBUuTs9WLox4gVDfWaCBpktx2fS2JeZS+6BsTV5bsiKyMNZIs +kK8fsCRXWMEwXsgqsPMJn/fd7R0Zr6SdmRVV022KqY9kJKJviRKQkQagpivv7koC +zgInEv8EdnomF5MNq2AyEOsgsnlu552hVVjQBTUnQFtSXa/mdnyjKJYWAHKXjEAG +DecE6iQRfXQ8DMWznHUodcD2XHp7jtquXDCXJ96rd/kooDSnEAb6rWeeBjRqvYdu +H9dfnKnKPNL+FmMa5l7vG10H4LV9O1Fb6pI2xuzuB7Wxg7zpcCju3dZDo+F1CXWW +8/YOZGhrnzeIZQbPSc2DTmXcsDcAYj8RxxO9R7dooX2wSkpfceGbHuWIT6k6GnFn +eRUVviH+RKR5McVvn+HutUtv/H55+ch6TF9th1zQZjlEd7wa5cX5I3ttmbhH+TXw +feWhGkWQAZl0GhynQFUHmgE/U7zzAgMBAAGgADANBgkqhkiG9w0BAQsFAAOCAgEA +Gewd4gSxvJJ1LIKLVTLeMdEdJfzsS52Kh3BCerku/C4ZKcIyT49dTbi6l6d40bHJ +Cs32Hwps8/qufhwwWn0r/wyww1Mgfr6ccMgKmrz1VbgWmD9owDzlL014ygiDk8yi +LwfSLe43NFrFC/FcAJUd/P8UEe0/8GcHjzhU/zqh3VAL7RzSL3k73CsTFiDaxtQL +8qISpA0lYTldMx5RaN9COWi0rPFv7mJAYRXtE/Cb+T2hb53khOiiRrJYIEJjjAhj +g8p9FOzlbXdpfoChVk7NA90CbCbxrQ2BiUqQAVYnGhntzHMwR9YxOYjvjnuiHuHX +7+4Vheda88TciMJlj0TC2e1mXBo182n/qfETeI26MsEOs3DidLT+ygM3woFQyIrX +3x2kDlvmILKg1mPGhqaRwwzCmk5a1TVdDTRo9VkOvR5/tbfG3vHlgpvBtUFCkEjX +HOyRh0A3TquofUbtx638lMWscBLqM5g6VO+Hytk6zBmq+8caJFNTOeTHZur04ZLM +SWfkIwl0B863owNuq4KxXI3NvpCc5LtGc9UrwVoHSH/pv6tbKEX15Y0ERL5/e33M +GT0D00cPnWAzmYQpYzHQ3Dj29XKlIxWBkn1QvrIFyb/T+dld1efZ3HlQxZEQvOsR +McY90r+HmVt8uCioYnC4DmchWlSX1MJe/h72udVbAXk= +-----END CERTIFICATE REQUEST----- diff --git a/tests/integration/test_ssl_cert_authentication/certs/generate_certs.sh b/tests/integration/test_ssl_cert_authentication/certs/generate_certs.sh index a09b7b2874e..e8d649fcff1 100755 --- a/tests/integration/test_ssl_cert_authentication/certs/generate_certs.sh +++ b/tests/integration/test_ssl_cert_authentication/certs/generate_certs.sh @@ -14,12 +14,16 @@ openssl req -newkey rsa:4096 -nodes -batch -keyout client1-key.pem -out client1- openssl req -newkey rsa:4096 -nodes -batch -keyout client2-key.pem -out client2-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client2" openssl req -newkey rsa:4096 -nodes -batch -keyout client3-key.pem -out client3-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client3" openssl req -newkey rsa:4096 -nodes -batch -keyout client4-key.pem -out client4-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client4" +openssl req -newkey rsa:4096 -nodes -batch -keyout client5-key.pem -out client5-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client5" +openssl req -newkey rsa:4096 -nodes -batch -keyout client6-key.pem -out client6-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client6" # 5. Use CA's private key to sign client's CSR and get back the signed certificate openssl x509 -req -days 3650 -in client1-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client1-cert.pem openssl x509 -req -days 3650 -in client2-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client2-cert.pem openssl x509 -req -days 3650 -in client3-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client3-cert.pem openssl x509 -req -days 3650 -in client4-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client4-ext.cnf -out client4-cert.pem +openssl x509 -req -days 3650 -in client5-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client5-ext.cnf -out client5-cert.pem +openssl x509 -req -days 3650 -in client6-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -extfile client6-ext.cnf -out client6-cert.pem # 6. Generate one more self-signed certificate and private key for using as wrong certificate (because it's not signed by CA) openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout wrong-key.pem -out wrong-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client" diff --git a/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml b/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml index 4bd30163ea6..b697c010195 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/users_with_ssl_auth.xml @@ -17,6 +17,11 @@ URI:spiffe://foo.com/baz + + + URI:spiffe://bar.com/foo/*/far + + diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 3af88759e82..4b5f3332a48 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -334,6 +334,8 @@ def test_create_user(): 'lucy\tssl_certificate\t{"common_names":["client2","client3"]}\n' ) + instance.query("DROP USER emma") + def test_x509_san_support(): assert ( @@ -369,3 +371,42 @@ def test_x509_san_support(): instance.query("SHOW CREATE USER jemma") == "CREATE USER jemma IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://foo.com/bar\\', \\'URI:spiffe://foo.com/baz\\'\n" ) + + instance.query("DROP USER jemma") + + +def test_x509_san_wildcard_support(): + assert ( + execute_query_native( + instance, "SELECT currentUser()", user="stewie", cert_name="client5" + ) + == "stewie\n" + ) + + assert ( + instance.query( + "SELECT name, auth_type, auth_params FROM system.users WHERE name='stewie'" + ) + == 'stewie\tssl_certificate\t{"subject_alt_names":["URI:spiffe:\\\\/\\\\/bar.com\\\\/foo\\\\/*\\\\/far"]}\n' + ) + + assert ( + instance.query("SHOW CREATE USER stewie") + == "CREATE USER stewie IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://bar.com/foo/*/far\\'\n" + ) + + instance.query( + "CREATE USER brian IDENTIFIED WITH ssl_certificate SAN 'URI:spiffe://bar.com/foo/*/far'" + ) + + assert ( + execute_query_https("SELECT currentUser()", user="brian", cert_name="client6") + == "brian\n" + ) + + assert ( + instance.query("SHOW CREATE USER brian") + == "CREATE USER brian IDENTIFIED WITH ssl_certificate SAN \\'URI:spiffe://bar.com/foo/*/far\\'\n" + ) + + instance.query("DROP USER brian") diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 9e3ee19179a..b75ad21f002 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -662,10 +662,7 @@ def test_multiple_tables_meta_mismatch(started_cluster): }, ) except QueryRuntimeException as e: - assert ( - "Table columns structure in ZooKeeper is different from local table structure" - in str(e) - ) + assert "Existing table metadata in ZooKeeper differs in columns" in str(e) failed = True assert failed is True diff --git a/tests/queries/0_stateless/01801_s3_cluster.reference b/tests/queries/0_stateless/01801_s3_cluster.reference index 4166d1718b1..c77baca9f09 100644 --- a/tests/queries/0_stateless/01801_s3_cluster.reference +++ b/tests/queries/0_stateless/01801_s3_cluster.reference @@ -190,3 +190,195 @@ 20 21 22 23 24 25 26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 +0 0 0 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +20 21 22 +23 24 25 +26 27 28 diff --git a/tests/queries/0_stateless/01801_s3_cluster.sql b/tests/queries/0_stateless/01801_s3_cluster.sql index 68d90ea4be0..f94f1102dc0 100644 --- a/tests/queries/0_stateless/01801_s3_cluster.sql +++ b/tests/queries/0_stateless/01801_s3_cluster.sql @@ -2,21 +2,37 @@ -- Tag no-fasttest: Depends on AWS select * from s3('http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN) ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV') ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; -select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '') ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV') ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3('http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN) ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; -select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', NOSIGN, 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64') ORDER BY c1, c2, c3; select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; +select * from s3Cluster('test_cluster_two_shards_localhost', 'http://localhost:11111/test/{a,b,c}.tsv', 'test', 'testtest', '', 'TSV', 'c1 UInt64, c2 UInt64, c3 UInt64', 'auto') ORDER BY c1, c2, c3; diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index 0409c48f846..e02684c3b95 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -100,7 +100,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off 504 2000 1000000 -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; (Expression) ExpressionTransform (Limit) @@ -119,6 +119,31 @@ ExpressionTransform Resize 1 → 4 GroupingAggregatedTransform 3 → 1 (ReadFromRemoteParallelReplicas) +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; +(Expression) +ExpressionTransform + (Limit) + Limit + (Sorting) + MergingSortedTransform 4 → 1 + MergeSortingTransform × 4 + LimitsCheckingTransform × 4 + PartialSortingTransform × 4 + (Expression) + ExpressionTransform × 4 + (MergingAggregated) + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + FinishAggregatingInOrderTransform 3 → 1 + (Union) + (Aggregating) + FinalizeAggregatedTransform + AggregatingInOrderTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolParallelReplicasInOrder, algorithm: InOrder) 0 → 1 + (ReadFromRemoteParallelReplicas) select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index a2de19dff8a..112640a2e2c 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -66,14 +66,14 @@ insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); set allow_experimental_parallel_reading_from_replicas = 1; set parallel_replicas_for_non_replicated_merge_tree = 1; set max_parallel_replicas = 3; -set use_hedged_requests = 0; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; set distributed_aggregation_memory_efficient=1; select count() from pr_t; -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; select a, count() from pr_t group by a order by a limit 5 offset 500; select a, count() from pr_t group by a, b order by a limit 5 offset 500; diff --git a/tests/queries/0_stateless/02494_query_cache_bugs.reference b/tests/queries/0_stateless/02494_query_cache_bugs.reference index 448e1366ea7..ea9017d5394 100644 --- a/tests/queries/0_stateless/02494_query_cache_bugs.reference +++ b/tests/queries/0_stateless/02494_query_cache_bugs.reference @@ -22,3 +22,24 @@ Row 1: ────── x: 1 2 +-- Bug 67476: Queries with overflow mode != throw must not be cached by the query cache +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02494_query_cache_bugs.sql b/tests/queries/0_stateless/02494_query_cache_bugs.sql index 74496e0f77a..423068aa646 100644 --- a/tests/queries/0_stateless/02494_query_cache_bugs.sql +++ b/tests/queries/0_stateless/02494_query_cache_bugs.sql @@ -36,4 +36,32 @@ SELECT count(*) FROM system.query_cache; DROP TABLE tab; +SELECT '-- Bug 67476: Queries with overflow mode != throw must not be cached by the query cache'; + +DROP TABLE IF EXISTS tab; + +CREATE TABLE tab(c UInt64) ENGINE = Memory; + +SYSTEM DROP QUERY CACHE; +SELECT sum(c) FROM tab SETTINGS read_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS read_overflow_mode_leaf = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS group_by_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS sort_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS result_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS timeout_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS set_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS join_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS transfer_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; +SELECT sum(c) FROM tab SETTINGS distinct_overflow_mode = 'break', use_query_cache = 1; +SELECT count(*) from system.query_cache; + SYSTEM DROP QUERY CACHE; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 125fa524c4a..682a43aab5f 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -31,7 +31,7 @@ 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== -0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') +0 2 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1; @@ -58,7 +58,7 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 t _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` +0 2 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` 1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index 8121d60a05b..299f1cbbc32 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -22,8 +22,8 @@ SELECT LIMIT 100; SET max_parallel_replicas = 3; -SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET parallel_replicas_local_plan = 1; SET joined_subquery_requires_alias = 0; SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 1a75e000349..7dbc389b55c 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -4,7 +4,9 @@ INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; -SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; +-- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected +-- therefore disable local plan for now +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', parallel_replicas_local_plan=0; SYSTEM FLUSH LOGS; diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index f60f1e0a376..3c54d5977f3 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -9,4 +9,4 @@ 7885388429666205427 8124171311239967992 1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; -0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` +0 1 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index e60049f2756..54295ffda17 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper - +DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated; CREATE TABLE join_inner_table__fuzz_146_replicated ( `id` UUID, @@ -17,6 +17,8 @@ INSERT INTO join_inner_table__fuzz_146_replicated SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10; +SET parallel_replicas_local_plan = 1; + -- Simple query with analyzer and pure parallel replicas SELECT number FROM join_inner_table__fuzz_146_replicated @@ -49,3 +51,5 @@ WHERE ) GROUP BY is_initial_query, query ORDER BY is_initial_query DESC, c, query; + +DROP TABLE join_inner_table__fuzz_146_replicated; diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 8a3b34e5cfa..ef41a5d6277 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -6,16 +6,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } @@ -48,11 +49,11 @@ function run_query_with_pure_parallel_replicas () { --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ - --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --max_threads 5 \ |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index 521e3e2edbc..595b35db610 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -4,8 +4,8 @@ 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas -02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_0_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Used parallel replicas: true diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index 801cd22b79f..238d63e6980 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -9,17 +9,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } diff --git a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh index c82d2c8b0c0..01e2883c547 100755 --- a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh +++ b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer query_id_base="02841_summary_$CLICKHOUSE_DATABASE" +# TODO: rethink the test, for now temporary disable parallel_replicas_local_plan echo " SELECT * FROM replicas_summary @@ -36,7 +37,8 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=0 + interactive_delay=0, + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' @@ -50,7 +52,8 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=99999999999 + interactive_delay=99999999999, + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index d8bfec12b3a..9348ea1dc32 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -26,7 +26,8 @@ WHERE query_id in (select query_id from system.query_log where current_database AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- reading in order coordinator -SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; +-- disable parallel_replicas_local_plan since the test relay on traces which only present in case of no local plan +SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; SELECT count() > 0 FROM system.text_log diff --git a/tests/queries/0_stateless/02933_sqid.reference b/tests/queries/0_stateless/02933_sqid.reference index a559bacb0ac..4597e2347e3 100644 --- a/tests/queries/0_stateless/02933_sqid.reference +++ b/tests/queries/0_stateless/02933_sqid.reference @@ -13,5 +13,6 @@ Td1EnWQo [1,2,3,4] XMbT -- invalid sqid [] +-- bug 69450 -- alias XMbT diff --git a/tests/queries/0_stateless/02933_sqid.sql b/tests/queries/0_stateless/02933_sqid.sql index 81d4b2bc35c..822fe33df51 100644 --- a/tests/queries/0_stateless/02933_sqid.sql +++ b/tests/queries/0_stateless/02933_sqid.sql @@ -25,5 +25,12 @@ SELECT sqidEncode(toNullable(materialize(1)), toLowCardinality(materialize(2))); SELECT '-- invalid sqid'; SELECT sqidDecode('invalid sqid'); +SELECT '-- bug 69450'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (id String) ENGINE = MergeTree ORDER BY id; +INSERT INTO tab SELECT * FROM generateRandom() LIMIT 1000000; +SELECT sqidDecode(id) FROM tab FORMAT Null; +DROP TABLE tab; + SELECT '-- alias'; SELECT sqid(1, 2); diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference index 21b7b527b7a..c1265b7ca14 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference @@ -1,4 +1,4 @@ -100 4950 +10000 49995000 1 89 90 @@ -6,3 +6,9 @@ 92 93 1 +93 +92 +91 +90 +89 +1 diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql index 22f55acd365..4396ca60e0e 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql @@ -2,11 +2,12 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (k UInt64, v String) ENGINE = MergeTree -ORDER BY k; +ORDER BY k +SETTINGS index_granularity=1; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); -SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; -- default coordinator SELECT count(), sum(k) @@ -14,12 +15,18 @@ FROM test SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count'; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- In order coordinator -SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2'; +SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; + +-- In reverse order coordinator +SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; DROP TABLE test; diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference index c0485b817c4..87e78d7d116 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.reference @@ -11,10 +11,8 @@ nested join with analyzer and parallel replicas, both global 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, global + local @@ -29,10 +27,8 @@ nested join with analyzer and parallel replicas, global + local 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting merge join @@ -47,10 +43,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting m 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -65,10 +59,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -83,8 +75,6 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh index c4b44ce11c5..a595e363ef4 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh @@ -17,6 +17,8 @@ insert into num_1 select number * 2, toString(number * 2) from numbers(1e7); insert into num_2 select number * 3, -number from numbers(1.5e6); " +PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan=1" + ############## echo echo "nested join with analyzer and parallel replicas, both global" @@ -26,18 +28,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -51,18 +49,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -77,18 +71,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -102,18 +92,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -127,18 +113,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 36f02b2f764..ba67acd7d0b 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -19,8 +19,20 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 14 14 14 14 0 0 15 15 0 0 0 0 explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -55,8 +67,22 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -80,8 +106,22 @@ Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -118,10 +158,22 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -160,12 +212,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -202,12 +266,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -242,14 +318,26 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -284,8 +372,28 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -333,10 +441,18 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas set parallel_replicas_prefer_local_join = 1; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; @@ -356,8 +472,22 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 14 14 14 14 0 0 15 15 0 0 0 0 explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -392,8 +522,24 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -417,8 +563,24 @@ Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -455,10 +617,23 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -497,12 +672,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -539,12 +726,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -579,14 +778,26 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -621,8 +832,30 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -670,7 +903,15 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 775663768fe..6f90f07c7a5 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -10,6 +10,8 @@ insert into tab1 select number, number, number from numbers(16); insert into tab2 select number * 2, number * 2 from numbers(8); insert into tab3 select number * 4, number * 4 from numbers(4); +set parallel_replicas_local_plan=1; + {% for use_global_in in [0, 1] -%} -- { echoOn } diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference index e69de29bb2d..8b8d0e5d565 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference @@ -0,0 +1 @@ +default url_na_log 1 130000 130 diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index 4b84646c034..de43302690a 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -21,11 +21,21 @@ SELECT FROM numbers(130000) SETTINGS max_insert_block_size = 200000; +SET max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; + EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; +SETTINGS parallel_replicas_local_plan=0; + +-- here parallel replicas uses local snapshot as working set +-- so, the estimation can be done +EXPLAIN ESTIMATE +SELECT count() +FROM url_na_log +PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') +SETTINGS allow_experimental_analyzer=1, parallel_replicas_local_plan=1; DROP POLICY url_na_log_policy0 ON url_na_log; DROP TABLE url_na_log; diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.reference b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference new file mode 100644 index 00000000000..619e98a152a --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference @@ -0,0 +1,4 @@ +key Sparse +value Sparse +1000 +1 diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.sql b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql new file mode 100644 index 00000000000..4de6d1ac6df --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t_bloom_filter; +CREATE TABLE t_bloom_filter( + key UInt64, + value UInt64, + + INDEX key_bf key TYPE bloom_filter(0.01) GRANULARITY 2147483648, -- bloom filter on sorting key column + INDEX value_bf value TYPE bloom_filter(0.01) GRANULARITY 2147483648 -- bloom filter on no-sorting column +) ENGINE=MergeTree ORDER BY key +SETTINGS + -- settings to trigger sparse serialization and vertical merge + ratio_of_defaults_for_sparse_serialization = 0.0 + ,vertical_merge_algorithm_min_rows_to_activate = 1 + ,vertical_merge_algorithm_min_columns_to_activate = 1 + ,allow_vertical_merges_from_compact_to_wide_parts = 1 + ,min_bytes_for_wide_part=0 +; + +SYSTEM STOP MERGES t_bloom_filter; + +-- Create at least one part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000); + +-- And another part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000, 50_000); + +SYSTEM START MERGES t_bloom_filter; + +-- Merge everything into a single part +OPTIMIZE TABLE t_bloom_filter FINAL; + +-- Check sparse serialization +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_bloom_filter' AND active ORDER BY column; + +SELECT COUNT() FROM t_bloom_filter WHERE key = 1; + +-- Check bloom filter non-zero size +SELECT COUNT() FROM system.parts WHERE database = currentDatabase() AND table = 't_bloom_filter' AND secondary_indices_uncompressed_bytes > 200 AND active; diff --git a/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql new file mode 100644 index 00000000000..6c2e05c12a9 --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS test_00808; + +CREATE TABLE test_00808 +( + `date` Date, + `id` Int8, + `name` String, + `value` Int64, + `sign` Int8 +) +ENGINE = CollapsingMergeTree(sign) +ORDER BY (id, date); + +INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1); +INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE test_00808; diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference new file mode 100644 index 00000000000..662880055bd --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference @@ -0,0 +1,100 @@ +0 18 9899 +0 18 9898 +0 18 9897 +0 18 9896 +0 18 9895 +0 18 9894 +0 18 9893 +0 18 9892 +0 18 9891 +0 18 9890 +0 18 9889 +0 18 9888 +0 18 9887 +0 18 9886 +0 18 9885 +0 18 9884 +0 18 9883 +0 18 9882 +0 18 9881 +0 18 9880 +0 18 9879 +0 18 9878 +0 18 9877 +0 18 9876 +0 18 9875 +0 18 9874 +0 18 9873 +0 18 9872 +0 18 9871 +0 18 9870 +0 18 9869 +0 18 9868 +0 18 9867 +0 18 9866 +0 18 9865 +0 18 9864 +0 18 9863 +0 18 9862 +0 18 9861 +0 18 9860 +0 18 9859 +0 18 9858 +0 18 9857 +0 18 9856 +0 18 9855 +0 18 9854 +0 18 9853 +0 18 9852 +0 18 9851 +0 18 9850 +0 18 9849 +0 18 9848 +0 18 9847 +0 18 9846 +0 18 9845 +0 18 9844 +0 18 9843 +0 18 9842 +0 18 9841 +0 18 9840 +0 18 9839 +0 18 9838 +0 18 9837 +0 18 9836 +0 18 9835 +0 18 9834 +0 18 9833 +0 18 9832 +0 18 9831 +0 18 9830 +0 18 9829 +0 18 9828 +0 18 9827 +0 18 9826 +0 18 9825 +0 18 9824 +0 18 9823 +0 18 9822 +0 18 9821 +0 18 9820 +0 18 9819 +0 18 9818 +0 18 9817 +0 18 9816 +0 18 9815 +0 18 9814 +0 18 9813 +0 18 9812 +0 18 9811 +0 18 9810 +0 18 9809 +0 18 9808 +0 18 9807 +0 18 9806 +0 18 9805 +0 18 9804 +0 18 9803 +0 18 9802 +0 18 9801 +0 18 9800 diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql new file mode 100644 index 00000000000..b7f37dd2856 --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (k UInt64, v String) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity=1; + +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT 0, materialize(18), k FROM test PREWHERE toNullable(toNullable(11)) WHERE toNullable(11) ORDER BY k DESC NULLS LAST LIMIT 100, 100 SETTINGS optimize_read_in_order = 1, merge_tree_min_rows_for_concurrent_read = 9223372036854775806, max_threads = 1; + +-- DROP TABLE test; diff --git a/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference b/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference new file mode 100644 index 00000000000..b62923296e5 --- /dev/null +++ b/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference @@ -0,0 +1,2 @@ +9 +9 diff --git a/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql new file mode 100644 index 00000000000..f3ee0f0b933 --- /dev/null +++ b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql @@ -0,0 +1,14 @@ +drop table if exists test_left; +drop table if exists test_right; + +CREATE TABLE test_left (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a; +CREATE TABLE test_right (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a; + +INSERT INTO test_left SELECT number % 10000, number % 10000, number % 10000 FROM numbers(100000); +INSERT INTO test_right SELECT number % 10 , number % 10, number % 10 FROM numbers(10000); + +SELECT MAX(test_right.a) FROM test_left INNER JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true; +SELECT MAX(test_right.a) FROM test_left LEFT JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true; + +drop table test_left; +drop table test_right; diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference new file mode 100644 index 00000000000..aad720e27cf --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference @@ -0,0 +1,20 @@ +300 +299 +298 +297 +296 +295 +294 +293 +292 +291 +290 +289 +288 +287 +286 +285 +284 +283 +282 +281 diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql new file mode 100644 index 00000000000..804a97f737f --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS view1; +DROP TABLE IF EXISTS table1; +CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1; +INSERT INTO table1 SELECT number FROM numbers(1, 300); +CREATE VIEW view1 AS SELECT number FROM table1; + +SELECT * +FROM +( + SELECT * + FROM view1 +) +ORDER BY number DESC +LIMIT 20 +SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1; + +DROP TABLE view1; +DROP TABLE table1; diff --git a/tests/queries/0_stateless/03235_groupArray_string_consistency.reference b/tests/queries/0_stateless/03235_groupArray_string_consistency.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03235_groupArray_string_consistency.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03235_groupArray_string_consistency.sql b/tests/queries/0_stateless/03235_groupArray_string_consistency.sql new file mode 100644 index 00000000000..618ec6f839b --- /dev/null +++ b/tests/queries/0_stateless/03235_groupArray_string_consistency.sql @@ -0,0 +1,10 @@ +CREATE TABLE t (st FixedString(54)) ENGINE=MergeTree ORDER BY (); + +INSERT INTO t VALUES +('abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRTUVWXYZ'), +('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'), +('IIIIIIIIII\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'); + +WITH (SELECT groupConcat(',')(st) FROM t) AS a, + (SELECT groupConcat(',')(st :: String) FROM t) AS b +SELECT equals(a, b); diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.reference b/tests/queries/0_stateless/03236_squashing_high_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql new file mode 100644 index 00000000000..f6e5dbdef03 --- /dev/null +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -0,0 +1,27 @@ +-- Tags: no-fasttest, no-asan, no-tsan, no-msan, no-ubsan +-- reason: test requires too many rows to read + +SET max_rows_to_read = '501G'; + +DROP TABLE IF EXISTS id_values; + +DROP TABLE IF EXISTS test_table; + +CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS + SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; + +SET max_memory_usage = '1G'; + +CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS +SELECT id_values.id1 AS id, + string_values.string_val1 AS string_val1, + string_values.string_val2 AS string_val2 +FROM id_values + JOIN (SELECT arrayJoin(range(10)) AS id1, + 'qwe' AS string_val1, + 'asd' AS string_val2) AS string_values + ON id_values.id1 = string_values.id1 + SETTINGS join_algorithm = 'hash'; + +DROP TABLE IF EXISTS id_values; +DROP TABLE IF EXISTS test_table; diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.reference b/tests/queries/1_stateful/00177_memory_bound_merging.reference index 5689152d60a..4c7505d1123 100644 --- a/tests/queries/1_stateful/00177_memory_bound_merging.reference +++ b/tests/queries/1_stateful/00177_memory_bound_merging.reference @@ -10,6 +10,8 @@ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-18 http:/ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-19 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-20 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 1 -SortingAggregatedTransform MergingAggregatedBucketTransform -GroupingAggregatedTransform +FinishAggregatingInOrderTransform +FinalizeAggregatedTransform +AggregatingInOrderTransform +MergeTreeSelect diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index 564160bb251..3744d89f354 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -55,7 +55,7 @@ test2() { test3() { $CLICKHOUSE_CLIENT -q " SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; - SET max_threads = 16, prefer_localhost_replica = 1, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; + SET max_threads = 16, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; SELECT replaceRegexpOne(explain, '^ *(\w+).*', '\\1') FROM ( @@ -64,7 +64,7 @@ test3() { FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3 + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, parallel_replicas_local_plan=1 ) WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'" } diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 796df6f1042..3467f21c812 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -24,6 +24,7 @@ Aggregatefunction AggregatingMergeTree AggregatorThreads AggregatorThreadsActive +AzureQueue Akka AlertManager Alexey diff --git a/utils/check-style/experimental_settings_ignore.txt b/utils/check-style/experimental_settings_ignore.txt index 94c46cf562e..3eda9821799 100644 --- a/utils/check-style/experimental_settings_ignore.txt +++ b/utils/check-style/experimental_settings_ignore.txt @@ -31,6 +31,7 @@ allow_experimental_statistics allow_experimental_time_series_table allow_experimental_undrop_table_query allow_experimental_usearch_index +allow_experimental_join_right_table_sorting allow_get_client_http_header allow_introspection_functions allow_materialized_view_with_bad_select